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