1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapreduce;
20
21 import static java.lang.String.format;
22
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.HashSet;
26 import java.util.Set;
27
28 import org.apache.commons.lang.StringUtils;
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.hbase.classification.InterfaceAudience;
32 import org.apache.hadoop.hbase.classification.InterfaceStability;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.conf.Configured;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.HBaseConfiguration;
37 import org.apache.hadoop.hbase.HColumnDescriptor;
38 import org.apache.hadoop.hbase.HConstants;
39 import org.apache.hadoop.hbase.HTableDescriptor;
40 import org.apache.hadoop.hbase.TableName;
41 import org.apache.hadoop.hbase.TableNotFoundException;
42 import org.apache.hadoop.hbase.client.HBaseAdmin;
43 import org.apache.hadoop.hbase.client.HTable;
44 import org.apache.hadoop.hbase.client.Put;
45 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
46 import org.apache.hadoop.hbase.util.Base64;
47 import org.apache.hadoop.hbase.util.Bytes;
48 import org.apache.hadoop.hbase.util.Pair;
49 import org.apache.hadoop.io.Text;
50 import org.apache.hadoop.mapreduce.Job;
51 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
52 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
53 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
54 import org.apache.hadoop.security.Credentials;
55 import org.apache.hadoop.util.GenericOptionsParser;
56 import org.apache.hadoop.util.Tool;
57 import org.apache.hadoop.util.ToolRunner;
58
59 import com.google.common.base.Preconditions;
60 import com.google.common.base.Splitter;
61 import com.google.common.collect.Lists;
62
63
64
65
66
67
68
69
70
71 @InterfaceAudience.Public
72 @InterfaceStability.Stable
73 public class ImportTsv extends Configured implements Tool {
74
75 protected static final Log LOG = LogFactory.getLog(ImportTsv.class);
76
77 final static String NAME = "importtsv";
78
79 public final static String MAPPER_CONF_KEY = "importtsv.mapper.class";
80 public final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
81 public final static String TIMESTAMP_CONF_KEY = "importtsv.timestamp";
82 public final static String JOB_NAME_CONF_KEY = "mapred.job.name";
83
84
85 public final static String CREDENTIALS_LOCATION = "credentials_location";
86
87
88 public final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
89 public final static String COLUMNS_CONF_KEY = "importtsv.columns";
90 public final static String SEPARATOR_CONF_KEY = "importtsv.separator";
91 public final static String ATTRIBUTE_SEPERATOR_CONF_KEY = "attributes.seperator";
92 final static String DEFAULT_SEPARATOR = "\t";
93 final static String DEFAULT_ATTRIBUTES_SEPERATOR = "=>";
94 final static String DEFAULT_MULTIPLE_ATTRIBUTES_SEPERATOR = ",";
95 final static Class DEFAULT_MAPPER = TsvImporterMapper.class;
96 public final static String CREATE_TABLE_CONF_KEY = "create.table";
97
98 public static class TsvParser {
99
100
101
102 private final byte[][] families;
103 private final byte[][] qualifiers;
104
105 private final byte separatorByte;
106
107 private int rowKeyColumnIndex;
108
109 private int maxColumnCount;
110
111
112 public static final int DEFAULT_TIMESTAMP_COLUMN_INDEX = -1;
113
114 private int timestampKeyColumnIndex = DEFAULT_TIMESTAMP_COLUMN_INDEX;
115
116 public static final String ROWKEY_COLUMN_SPEC = "HBASE_ROW_KEY";
117
118 public static final String TIMESTAMPKEY_COLUMN_SPEC = "HBASE_TS_KEY";
119
120 public static final String ATTRIBUTES_COLUMN_SPEC = "HBASE_ATTRIBUTES_KEY";
121
122 @InterfaceStability.Unstable
123 public static final String CELL_VISIBILITY_COLUMN_SPEC = "HBASE_CELL_VISIBILITY";
124 @InterfaceStability.Unstable
125 public static final String CELL_TTL_COLUMN_SPEC = "HBASE_CELL_TTL";
126
127 private int attrKeyColumnIndex = DEFAULT_ATTRIBUTES_COLUMN_INDEX;
128
129 public static final int DEFAULT_ATTRIBUTES_COLUMN_INDEX = -1;
130
131 @InterfaceStability.Unstable
132 public static final int DEFAULT_CELL_VISIBILITY_COLUMN_INDEX = -1;
133 @InterfaceStability.Unstable
134 public static final int DEFAULT_CELL_TTL_COLUMN_INDEX = -1;
135
136 private int cellVisibilityColumnIndex = DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
137
138 private int cellTTLColumnIndex = DEFAULT_CELL_TTL_COLUMN_INDEX;
139
140
141
142
143
144
145 public TsvParser(String columnsSpecification, String separatorStr) {
146
147 byte[] separator = Bytes.toBytes(separatorStr);
148 Preconditions.checkArgument(separator.length == 1,
149 "TsvParser only supports single-byte separators");
150 separatorByte = separator[0];
151
152
153 ArrayList<String> columnStrings = Lists.newArrayList(
154 Splitter.on(',').trimResults().split(columnsSpecification));
155
156 maxColumnCount = columnStrings.size();
157 families = new byte[maxColumnCount][];
158 qualifiers = new byte[maxColumnCount][];
159
160 for (int i = 0; i < columnStrings.size(); i++) {
161 String str = columnStrings.get(i);
162 if (ROWKEY_COLUMN_SPEC.equals(str)) {
163 rowKeyColumnIndex = i;
164 continue;
165 }
166 if (TIMESTAMPKEY_COLUMN_SPEC.equals(str)) {
167 timestampKeyColumnIndex = i;
168 continue;
169 }
170 if (ATTRIBUTES_COLUMN_SPEC.equals(str)) {
171 attrKeyColumnIndex = i;
172 continue;
173 }
174 if (CELL_VISIBILITY_COLUMN_SPEC.equals(str)) {
175 cellVisibilityColumnIndex = i;
176 continue;
177 }
178 if (CELL_TTL_COLUMN_SPEC.equals(str)) {
179 cellTTLColumnIndex = i;
180 continue;
181 }
182 String[] parts = str.split(":", 2);
183 if (parts.length == 1) {
184 families[i] = str.getBytes();
185 qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
186 } else {
187 families[i] = parts[0].getBytes();
188 qualifiers[i] = parts[1].getBytes();
189 }
190 }
191 }
192
193 public boolean hasTimestamp() {
194 return timestampKeyColumnIndex != DEFAULT_TIMESTAMP_COLUMN_INDEX;
195 }
196
197 public int getTimestampKeyColumnIndex() {
198 return timestampKeyColumnIndex;
199 }
200
201 public boolean hasAttributes() {
202 return attrKeyColumnIndex != DEFAULT_ATTRIBUTES_COLUMN_INDEX;
203 }
204
205 @InterfaceStability.Unstable
206 public boolean hasCellVisibility() {
207 return cellVisibilityColumnIndex != DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
208 }
209 @InterfaceStability.Unstable
210 public boolean hasCellTTL() {
211 return cellTTLColumnIndex != DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
212 }
213
214 public int getAttributesKeyColumnIndex() {
215 return attrKeyColumnIndex;
216 }
217 @InterfaceStability.Unstable
218 public int getCellVisibilityColumnIndex() {
219 return cellVisibilityColumnIndex;
220 }
221 @InterfaceStability.Unstable
222 public int getCellTTLColumnIndex() {
223 return cellTTLColumnIndex;
224 }
225
226 public int getRowKeyColumnIndex() {
227 return rowKeyColumnIndex;
228 }
229
230 public byte[] getFamily(int idx) {
231 return families[idx];
232 }
233 public byte[] getQualifier(int idx) {
234 return qualifiers[idx];
235 }
236
237 public ParsedLine parse(byte[] lineBytes, int length)
238 throws BadTsvLineException {
239
240 ArrayList<Integer> tabOffsets = new ArrayList<Integer>(maxColumnCount);
241 for (int i = 0; i < length; i++) {
242 if (lineBytes[i] == separatorByte) {
243 tabOffsets.add(i);
244 }
245 }
246 if (tabOffsets.isEmpty()) {
247 throw new BadTsvLineException("No delimiter");
248 }
249
250 tabOffsets.add(length);
251
252 if (tabOffsets.size() > maxColumnCount) {
253 throw new BadTsvLineException("Excessive columns");
254 } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
255 throw new BadTsvLineException("No row key");
256 } else if (hasTimestamp()
257 && tabOffsets.size() <= getTimestampKeyColumnIndex()) {
258 throw new BadTsvLineException("No timestamp");
259 } else if (hasAttributes() && tabOffsets.size() <= getAttributesKeyColumnIndex()) {
260 throw new BadTsvLineException("No attributes specified");
261 } else if (hasCellVisibility() && tabOffsets.size() <= getCellVisibilityColumnIndex()) {
262 throw new BadTsvLineException("No cell visibility specified");
263 } else if (hasCellTTL() && tabOffsets.size() <= getCellTTLColumnIndex()) {
264 throw new BadTsvLineException("No cell TTL specified");
265 }
266 return new ParsedLine(tabOffsets, lineBytes);
267 }
268
269 class ParsedLine {
270 private final ArrayList<Integer> tabOffsets;
271 private byte[] lineBytes;
272
273 ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
274 this.tabOffsets = tabOffsets;
275 this.lineBytes = lineBytes;
276 }
277
278 public int getRowKeyOffset() {
279 return getColumnOffset(rowKeyColumnIndex);
280 }
281 public int getRowKeyLength() {
282 return getColumnLength(rowKeyColumnIndex);
283 }
284
285 public long getTimestamp(long ts) throws BadTsvLineException {
286
287 if (!hasTimestamp()) {
288 return ts;
289 }
290
291 String timeStampStr = Bytes.toString(lineBytes,
292 getColumnOffset(timestampKeyColumnIndex),
293 getColumnLength(timestampKeyColumnIndex));
294 try {
295 return Long.parseLong(timeStampStr);
296 } catch (NumberFormatException nfe) {
297
298 throw new BadTsvLineException("Invalid timestamp " + timeStampStr);
299 }
300 }
301
302 private String getAttributes() {
303 if (!hasAttributes()) {
304 return null;
305 } else {
306 return Bytes.toString(lineBytes, getColumnOffset(attrKeyColumnIndex),
307 getColumnLength(attrKeyColumnIndex));
308 }
309 }
310
311 public String[] getIndividualAttributes() {
312 String attributes = getAttributes();
313 if (attributes != null) {
314 return attributes.split(DEFAULT_MULTIPLE_ATTRIBUTES_SEPERATOR);
315 } else {
316 return null;
317 }
318 }
319
320 public int getAttributeKeyOffset() {
321 if (hasAttributes()) {
322 return getColumnOffset(attrKeyColumnIndex);
323 } else {
324 return DEFAULT_ATTRIBUTES_COLUMN_INDEX;
325 }
326 }
327
328 public int getAttributeKeyLength() {
329 if (hasAttributes()) {
330 return getColumnLength(attrKeyColumnIndex);
331 } else {
332 return DEFAULT_ATTRIBUTES_COLUMN_INDEX;
333 }
334 }
335
336 public int getCellVisibilityColumnOffset() {
337 if (hasCellVisibility()) {
338 return getColumnOffset(cellVisibilityColumnIndex);
339 } else {
340 return DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
341 }
342 }
343
344 public int getCellVisibilityColumnLength() {
345 if (hasCellVisibility()) {
346 return getColumnLength(cellVisibilityColumnIndex);
347 } else {
348 return DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
349 }
350 }
351
352 public String getCellVisibility() {
353 if (!hasCellVisibility()) {
354 return null;
355 } else {
356 return Bytes.toString(lineBytes, getColumnOffset(cellVisibilityColumnIndex),
357 getColumnLength(cellVisibilityColumnIndex));
358 }
359 }
360
361 public int getCellTTLColumnOffset() {
362 if (hasCellTTL()) {
363 return getColumnOffset(cellTTLColumnIndex);
364 } else {
365 return DEFAULT_CELL_TTL_COLUMN_INDEX;
366 }
367 }
368
369 public int getCellTTLColumnLength() {
370 if (hasCellTTL()) {
371 return getColumnLength(cellTTLColumnIndex);
372 } else {
373 return DEFAULT_CELL_TTL_COLUMN_INDEX;
374 }
375 }
376
377 public long getCellTTL() {
378 if (!hasCellTTL()) {
379 return 0;
380 } else {
381 return Bytes.toLong(lineBytes, getColumnOffset(cellTTLColumnIndex),
382 getColumnLength(cellTTLColumnIndex));
383 }
384 }
385
386 public int getColumnOffset(int idx) {
387 if (idx > 0)
388 return tabOffsets.get(idx - 1) + 1;
389 else
390 return 0;
391 }
392 public int getColumnLength(int idx) {
393 return tabOffsets.get(idx) - getColumnOffset(idx);
394 }
395 public int getColumnCount() {
396 return tabOffsets.size();
397 }
398 public byte[] getLineBytes() {
399 return lineBytes;
400 }
401 }
402
403 public static class BadTsvLineException extends Exception {
404 public BadTsvLineException(String err) {
405 super(err);
406 }
407 private static final long serialVersionUID = 1L;
408 }
409
410
411
412
413
414
415
416
417 public Pair<Integer, Integer> parseRowKey(byte[] lineBytes, int length)
418 throws BadTsvLineException {
419 int rkColumnIndex = 0;
420 int startPos = 0, endPos = 0;
421 for (int i = 0; i <= length; i++) {
422 if (i == length || lineBytes[i] == separatorByte) {
423 endPos = i - 1;
424 if (rkColumnIndex++ == getRowKeyColumnIndex()) {
425 if ((endPos + 1) == startPos) {
426 throw new BadTsvLineException("Empty value for ROW KEY.");
427 }
428 break;
429 } else {
430 startPos = endPos + 2;
431 }
432 }
433 if (i == length) {
434 throw new BadTsvLineException(
435 "Row key does not exist as number of columns in the line"
436 + " are less than row key position.");
437 }
438 }
439 return new Pair<Integer, Integer>(startPos, endPos - startPos + 1);
440 }
441 }
442
443
444
445
446
447
448
449
450
451 public static Job createSubmittableJob(Configuration conf, String[] args)
452 throws IOException, ClassNotFoundException {
453
454 HBaseAdmin admin = new HBaseAdmin(conf);
455
456
457 String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
458 if (actualSeparator != null) {
459 conf.set(SEPARATOR_CONF_KEY,
460 Base64.encodeBytes(actualSeparator.getBytes()));
461 }
462
463
464 String mapperClassName = conf.get(MAPPER_CONF_KEY);
465 Class mapperClass = mapperClassName != null ?
466 Class.forName(mapperClassName) : DEFAULT_MAPPER;
467
468 String tableName = args[0];
469 Path inputDir = new Path(args[1]);
470 String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName);
471 Job job = new Job(conf, jobName);
472 job.setJarByClass(mapperClass);
473 FileInputFormat.setInputPaths(job, inputDir);
474 job.setInputFormatClass(TextInputFormat.class);
475 job.setMapperClass(mapperClass);
476 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
477 String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
478 if(StringUtils.isNotEmpty(conf.get(CREDENTIALS_LOCATION))) {
479 String fileLoc = conf.get(CREDENTIALS_LOCATION);
480 Credentials cred = Credentials.readTokenStorageFile(new Path(fileLoc), conf);
481 job.getCredentials().addAll(cred);
482 }
483
484 if (hfileOutPath != null) {
485 if (!admin.tableExists(tableName)) {
486 String errorMsg = format("Table '%s' does not exist.", tableName);
487 if ("yes".equalsIgnoreCase(conf.get(CREATE_TABLE_CONF_KEY, "yes"))) {
488 LOG.warn(errorMsg);
489
490
491 createTable(admin, tableName, columns);
492 } else {
493 LOG.error(errorMsg);
494 throw new TableNotFoundException(errorMsg);
495 }
496 }
497 HTable table = new HTable(conf, tableName);
498 job.setReducerClass(PutSortReducer.class);
499 Path outputDir = new Path(hfileOutPath);
500 FileOutputFormat.setOutputPath(job, outputDir);
501 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
502 if (mapperClass.equals(TsvImporterTextMapper.class)) {
503 job.setMapOutputValueClass(Text.class);
504 job.setReducerClass(TextSortReducer.class);
505 } else {
506 job.setMapOutputValueClass(Put.class);
507 job.setCombinerClass(PutCombiner.class);
508 }
509 HFileOutputFormat.configureIncrementalLoad(job, table);
510 } else {
511 if (!admin.tableExists(tableName)) {
512 String errorMsg = format("Table '%s' does not exist.", tableName);
513 LOG.error(errorMsg);
514 throw new TableNotFoundException(errorMsg);
515 }
516 if (mapperClass.equals(TsvImporterTextMapper.class)) {
517 usage(TsvImporterTextMapper.class.toString()
518 + " should not be used for non bulkloading case. use "
519 + TsvImporterMapper.class.toString()
520 + " or custom mapper whose value type is Put.");
521 System.exit(-1);
522 }
523
524
525 TableMapReduceUtil.initTableReducerJob(tableName, null, job);
526 job.setNumReduceTasks(0);
527 }
528
529 TableMapReduceUtil.addDependencyJars(job);
530 TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
531 com.google.common.base.Function.class
532 return job;
533 }
534
535 private static void createTable(HBaseAdmin admin, String tableName, String[] columns)
536 throws IOException {
537 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
538 Set<String> cfSet = new HashSet<String>();
539 for (String aColumn : columns) {
540 if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)
541 || TsvParser.TIMESTAMPKEY_COLUMN_SPEC.equals(aColumn)
542 || TsvParser.CELL_VISIBILITY_COLUMN_SPEC.equals(aColumn)
543 || TsvParser.CELL_TTL_COLUMN_SPEC.equals(aColumn)
544 || TsvParser.ATTRIBUTES_COLUMN_SPEC.equals(aColumn))
545 continue;
546
547 cfSet.add(aColumn.split(":", 2)[0]);
548 }
549 for (String cf : cfSet) {
550 HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(cf));
551 htd.addFamily(hcd);
552 }
553 LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.",
554 tableName, cfSet));
555 admin.createTable(htd);
556 }
557
558
559
560
561 private static void usage(final String errorMsg) {
562 if (errorMsg != null && errorMsg.length() > 0) {
563 System.err.println("ERROR: " + errorMsg);
564 }
565 String usage =
566 "Usage: " + NAME + " -D"+ COLUMNS_CONF_KEY + "=a,b,c <tablename> <inputdir>\n" +
567 "\n" +
568 "Imports the given input directory of TSV data into the specified table.\n" +
569 "\n" +
570 "The column names of the TSV data must be specified using the -D" + COLUMNS_CONF_KEY + "\n" +
571 "option. This option takes the form of comma-separated column names, where each\n" +
572 "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
573 "column name " + TsvParser.ROWKEY_COLUMN_SPEC + " is used to designate that this column should be used\n" +
574 "as the row key for each imported record. You must specify exactly one column\n" +
575 "to be the row key, and you must specify a column name for every column that exists in the\n" +
576 "input data. Another special column" + TsvParser.TIMESTAMPKEY_COLUMN_SPEC +
577 " designates that this column should be\n" +
578 "used as timestamp for each record. Unlike " + TsvParser.ROWKEY_COLUMN_SPEC + ", " +
579 TsvParser.TIMESTAMPKEY_COLUMN_SPEC + " is optional.\n" +
580 "You must specify at most one column as timestamp key for each imported record.\n" +
581 "Record with invalid timestamps (blank, non-numeric) will be treated as bad record.\n" +
582 "Note: if you use this option, then '" + TIMESTAMP_CONF_KEY + "' option will be ignored.\n" +
583 "\n" +
584 TsvParser.ATTRIBUTES_COLUMN_SPEC+" can be used to specify Operation Attributes per record.\n"+
585 " Should be specified as key=>value where "+TsvParser.DEFAULT_ATTRIBUTES_COLUMN_INDEX+ " is used \n"+
586 " as the seperator. Note that more than one OperationAttributes can be specified.\n"+
587 "By default importtsv will load data directly into HBase. To instead generate\n" +
588 "HFiles of data to prepare for a bulk data load, pass the option:\n" +
589 " -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
590 " Note: if you do not use this option, then the target table must already exist in HBase\n" +
591 "\n" +
592 "Other options that may be specified with -D include:\n" +
593 " -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
594 " '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs\n" +
595 " -D" + TIMESTAMP_CONF_KEY + "=currentTimeAsLong - use the specified timestamp for the import\n" +
596 " -D" + MAPPER_CONF_KEY + "=my.Mapper - A user-defined Mapper to use instead of " +
597 DEFAULT_MAPPER.getName() + "\n" +
598 " -D" + JOB_NAME_CONF_KEY + "=jobName - use the specified mapreduce job name for the import\n" +
599 " -D" + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by this tool\n" +
600 " Note: if you set this to 'no', then the target table must already exist in HBase\n" +
601 "\n" +
602 "For performance consider the following options:\n" +
603 " -Dmapred.map.tasks.speculative.execution=false\n" +
604 " -Dmapred.reduce.tasks.speculative.execution=false";
605
606 System.err.println(usage);
607 }
608
609 @Override
610 public int run(String[] args) throws Exception {
611 setConf(HBaseConfiguration.create(getConf()));
612 String[] otherArgs = new GenericOptionsParser(getConf(), args).getRemainingArgs();
613 if (otherArgs.length < 2) {
614 usage("Wrong number of arguments: " + otherArgs.length);
615 return -1;
616 }
617
618
619
620
621
622 if (null == getConf().get(MAPPER_CONF_KEY)) {
623
624 String columns[] = getConf().getStrings(COLUMNS_CONF_KEY);
625 if (columns == null) {
626 usage("No columns specified. Please specify with -D" +
627 COLUMNS_CONF_KEY+"=...");
628 return -1;
629 }
630
631
632 int rowkeysFound = 0;
633 for (String col : columns) {
634 if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
635 }
636 if (rowkeysFound != 1) {
637 usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
638 return -1;
639 }
640
641
642 int tskeysFound = 0;
643 for (String col : columns) {
644 if (col.equals(TsvParser.TIMESTAMPKEY_COLUMN_SPEC))
645 tskeysFound++;
646 }
647 if (tskeysFound > 1) {
648 usage("Must specify at most one column as "
649 + TsvParser.TIMESTAMPKEY_COLUMN_SPEC);
650 return -1;
651 }
652
653 int attrKeysFound = 0;
654 for (String col : columns) {
655 if (col.equals(TsvParser.ATTRIBUTES_COLUMN_SPEC))
656 attrKeysFound++;
657 }
658 if (attrKeysFound > 1) {
659 usage("Must specify at most one column as "
660 + TsvParser.ATTRIBUTES_COLUMN_SPEC);
661 return -1;
662 }
663
664
665
666 if (columns.length - (rowkeysFound + tskeysFound + attrKeysFound) < 1) {
667 usage("One or more columns in addition to the row key and timestamp(optional) are required");
668 return -1;
669 }
670 }
671
672
673 long timstamp = getConf().getLong(TIMESTAMP_CONF_KEY, System.currentTimeMillis());
674
675
676
677 getConf().setLong(TIMESTAMP_CONF_KEY, timstamp);
678
679 Job job = createSubmittableJob(getConf(), otherArgs);
680 return job.waitForCompletion(true) ? 0 : 1;
681 }
682
683 public static void main(String[] args) throws Exception {
684 int status = ToolRunner.run(new ImportTsv(), args);
685 System.exit(status);
686 }
687 }