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