View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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   * Tool to import data from a TSV file.
65   *
66   * This tool is rather simplistic - it doesn't do any quoting or
67   * escaping, but is useful for many data loads.
68   *
69   * @see ImportTsv#usage(String)
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    //This config is used to propagate credentials from parent MR jobs which launch
84    //ImportTSV jobs. SEE IntegrationTestImportTsv.
85    public final static String CREDENTIALS_LOCATION = "credentials_location";
86    // TODO: the rest of these configs are used exclusively by TsvImporterMapper.
87    // Move them out of the tool and let the mapper handle its own validation.
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      * Column families and qualifiers mapped to the TSV columns
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     // Default value must be negative
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     public static final String CELL_VISIBILITY_COLUMN_SPEC = "HBASE_CELL_VISIBILITY";
123 
124     public static final String CELL_TTL_COLUMN_SPEC = "HBASE_CELL_TTL";
125 
126     private int attrKeyColumnIndex = DEFAULT_ATTRIBUTES_COLUMN_INDEX;
127 
128     public static final int DEFAULT_ATTRIBUTES_COLUMN_INDEX = -1;
129 
130     public static final int DEFAULT_CELL_VISIBILITY_COLUMN_INDEX = -1;
131 
132     public static final int DEFAULT_CELL_TTL_COLUMN_INDEX = -1;
133 
134     private int cellVisibilityColumnIndex = DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
135 
136     private int cellTTLColumnIndex = DEFAULT_CELL_TTL_COLUMN_INDEX;
137 
138     /**
139      * @param columnsSpecification the list of columns to parser out, comma separated.
140      * The row key should be the special token TsvParser.ROWKEY_COLUMN_SPEC
141      * @param separatorStr
142      */
143     public TsvParser(String columnsSpecification, String separatorStr) {
144       // Configure separator
145       byte[] separator = Bytes.toBytes(separatorStr);
146       Preconditions.checkArgument(separator.length == 1,
147         "TsvParser only supports single-byte separators");
148       separatorByte = separator[0];
149 
150       // Configure columns
151       ArrayList<String> columnStrings = Lists.newArrayList(
152         Splitter.on(',').trimResults().split(columnsSpecification));
153 
154       maxColumnCount = columnStrings.size();
155       families = new byte[maxColumnCount][];
156       qualifiers = new byte[maxColumnCount][];
157 
158       for (int i = 0; i < columnStrings.size(); i++) {
159         String str = columnStrings.get(i);
160         if (ROWKEY_COLUMN_SPEC.equals(str)) {
161           rowKeyColumnIndex = i;
162           continue;
163         }
164         if (TIMESTAMPKEY_COLUMN_SPEC.equals(str)) {
165           timestampKeyColumnIndex = i;
166           continue;
167         }
168         if (ATTRIBUTES_COLUMN_SPEC.equals(str)) {
169           attrKeyColumnIndex = i;
170           continue;
171         }
172         if (CELL_VISIBILITY_COLUMN_SPEC.equals(str)) {
173           cellVisibilityColumnIndex = i;
174           continue;
175         }
176         if (CELL_TTL_COLUMN_SPEC.equals(str)) {
177           cellTTLColumnIndex = i;
178           continue;
179         }
180         String[] parts = str.split(":", 2);
181         if (parts.length == 1) {
182           families[i] = str.getBytes();
183           qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
184         } else {
185           families[i] = parts[0].getBytes();
186           qualifiers[i] = parts[1].getBytes();
187         }
188       }
189     }
190 
191     public boolean hasTimestamp() {
192       return timestampKeyColumnIndex != DEFAULT_TIMESTAMP_COLUMN_INDEX;
193     }
194 
195     public int getTimestampKeyColumnIndex() {
196       return timestampKeyColumnIndex;
197     }
198 
199     public boolean hasAttributes() {
200       return attrKeyColumnIndex != DEFAULT_ATTRIBUTES_COLUMN_INDEX;
201     }
202 
203     public boolean hasCellVisibility() {
204       return cellVisibilityColumnIndex != DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
205     }
206 
207     public boolean hasCellTTL() {
208       return cellTTLColumnIndex != DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
209     }
210 
211     public int getAttributesKeyColumnIndex() {
212       return attrKeyColumnIndex;
213     }
214 
215     public int getCellVisibilityColumnIndex() {
216       return cellVisibilityColumnIndex;
217     }
218 
219     public int getCellTTLColumnIndex() {
220       return cellTTLColumnIndex;
221     }
222 
223     public int getRowKeyColumnIndex() {
224       return rowKeyColumnIndex;
225     }
226 
227     public byte[] getFamily(int idx) {
228       return families[idx];
229     }
230     public byte[] getQualifier(int idx) {
231       return qualifiers[idx];
232     }
233 
234     public ParsedLine parse(byte[] lineBytes, int length)
235     throws BadTsvLineException {
236       // Enumerate separator offsets
237       ArrayList<Integer> tabOffsets = new ArrayList<Integer>(maxColumnCount);
238       for (int i = 0; i < length; i++) {
239         if (lineBytes[i] == separatorByte) {
240           tabOffsets.add(i);
241         }
242       }
243       if (tabOffsets.isEmpty()) {
244         throw new BadTsvLineException("No delimiter");
245       }
246 
247       tabOffsets.add(length);
248 
249       if (tabOffsets.size() > maxColumnCount) {
250         throw new BadTsvLineException("Excessive columns");
251       } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
252         throw new BadTsvLineException("No row key");
253       } else if (hasTimestamp()
254           && tabOffsets.size() <= getTimestampKeyColumnIndex()) {
255         throw new BadTsvLineException("No timestamp");
256       } else if (hasAttributes() && tabOffsets.size() <= getAttributesKeyColumnIndex()) {
257         throw new BadTsvLineException("No attributes specified");
258       } else if (hasCellVisibility() && tabOffsets.size() <= getCellVisibilityColumnIndex()) {
259         throw new BadTsvLineException("No cell visibility specified");
260       } else if (hasCellTTL() && tabOffsets.size() <= getCellTTLColumnIndex()) {
261         throw new BadTsvLineException("No cell TTL specified");
262       }
263       return new ParsedLine(tabOffsets, lineBytes);
264     }
265 
266     class ParsedLine {
267       private final ArrayList<Integer> tabOffsets;
268       private byte[] lineBytes;
269 
270       ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
271         this.tabOffsets = tabOffsets;
272         this.lineBytes = lineBytes;
273       }
274 
275       public int getRowKeyOffset() {
276         return getColumnOffset(rowKeyColumnIndex);
277       }
278       public int getRowKeyLength() {
279         return getColumnLength(rowKeyColumnIndex);
280       }
281 
282       public long getTimestamp(long ts) throws BadTsvLineException {
283         // Return ts if HBASE_TS_KEY is not configured in column spec
284         if (!hasTimestamp()) {
285           return ts;
286         }
287 
288         String timeStampStr = Bytes.toString(lineBytes,
289             getColumnOffset(timestampKeyColumnIndex),
290             getColumnLength(timestampKeyColumnIndex));
291         try {
292           return Long.parseLong(timeStampStr);
293         } catch (NumberFormatException nfe) {
294           // treat this record as bad record
295           throw new BadTsvLineException("Invalid timestamp " + timeStampStr);
296         }
297       }
298 
299       private String getAttributes() {
300         if (!hasAttributes()) {
301           return null;
302         } else {
303           return Bytes.toString(lineBytes, getColumnOffset(attrKeyColumnIndex),
304               getColumnLength(attrKeyColumnIndex));
305         }
306       }
307 
308       public String[] getIndividualAttributes() {
309         String attributes = getAttributes();
310         if (attributes != null) {
311           return attributes.split(DEFAULT_MULTIPLE_ATTRIBUTES_SEPERATOR);
312         } else {
313           return null;
314         }
315       }
316 
317       public int getAttributeKeyOffset() {
318         if (hasAttributes()) {
319           return getColumnOffset(attrKeyColumnIndex);
320         } else {
321           return DEFAULT_ATTRIBUTES_COLUMN_INDEX;
322         }
323       }
324 
325       public int getAttributeKeyLength() {
326         if (hasAttributes()) {
327           return getColumnLength(attrKeyColumnIndex);
328         } else {
329           return DEFAULT_ATTRIBUTES_COLUMN_INDEX;
330         }
331       }
332 
333       public int getCellVisibilityColumnOffset() {
334         if (hasCellVisibility()) {
335           return getColumnOffset(cellVisibilityColumnIndex);
336         } else {
337           return DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
338         }
339       }
340 
341       public int getCellVisibilityColumnLength() {
342         if (hasCellVisibility()) {
343           return getColumnLength(cellVisibilityColumnIndex);
344         } else {
345           return DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
346         }
347       }
348 
349       public String getCellVisibility() {
350         if (!hasCellVisibility()) {
351           return null;
352         } else {
353           return Bytes.toString(lineBytes, getColumnOffset(cellVisibilityColumnIndex),
354               getColumnLength(cellVisibilityColumnIndex));
355         }
356       }
357 
358       public int getCellTTLColumnOffset() {
359         if (hasCellTTL()) {
360           return getColumnOffset(cellTTLColumnIndex);
361         } else {
362           return DEFAULT_CELL_TTL_COLUMN_INDEX;
363         }
364       }
365 
366       public int getCellTTLColumnLength() {
367         if (hasCellTTL()) {
368           return getColumnLength(cellTTLColumnIndex);
369         } else {
370           return DEFAULT_CELL_TTL_COLUMN_INDEX;
371         }
372       }
373 
374       public long getCellTTL() {
375         if (!hasCellTTL()) {
376           return 0;
377         } else {
378           return Bytes.toLong(lineBytes, getColumnOffset(cellTTLColumnIndex),
379               getColumnLength(cellTTLColumnIndex));
380         }
381       }
382 
383       public int getColumnOffset(int idx) {
384         if (idx > 0)
385           return tabOffsets.get(idx - 1) + 1;
386         else
387           return 0;
388       }
389       public int getColumnLength(int idx) {
390         return tabOffsets.get(idx) - getColumnOffset(idx);
391       }
392       public int getColumnCount() {
393         return tabOffsets.size();
394       }
395       public byte[] getLineBytes() {
396         return lineBytes;
397       }
398     }
399 
400     public static class BadTsvLineException extends Exception {
401       public BadTsvLineException(String err) {
402         super(err);
403       }
404       private static final long serialVersionUID = 1L;
405     }
406 
407     /**
408      * Return starting position and length of row key from the specified line bytes.
409      * @param lineBytes
410      * @param length
411      * @return Pair of row key offset and length.
412      * @throws BadTsvLineException
413      */
414     public Pair<Integer, Integer> parseRowKey(byte[] lineBytes, int length)
415         throws BadTsvLineException {
416       int rkColumnIndex = 0;
417       int startPos = 0, endPos = 0;
418       for (int i = 0; i <= length; i++) {
419         if (i == length || lineBytes[i] == separatorByte) {
420           endPos = i - 1;
421           if (rkColumnIndex++ == getRowKeyColumnIndex()) {
422             if ((endPos + 1) == startPos) {
423               throw new BadTsvLineException("Empty value for ROW KEY.");
424             }
425             break;
426           } else {
427             startPos = endPos + 2;
428           }
429         }
430         if (i == length) {
431           throw new BadTsvLineException(
432               "Row key does not exist as number of columns in the line"
433                   + " are less than row key position.");
434         }
435       }
436       return new Pair<Integer, Integer>(startPos, endPos - startPos + 1);
437     }
438   }
439 
440   /**
441    * Sets up the actual job.
442    *
443    * @param conf  The current configuration.
444    * @param args  The command line parameters.
445    * @return The newly created job.
446    * @throws IOException When setting up the job fails.
447    */
448   public static Job createSubmittableJob(Configuration conf, String[] args)
449       throws IOException, ClassNotFoundException {
450 
451     HBaseAdmin admin = new HBaseAdmin(conf);
452     // Support non-XML supported characters
453     // by re-encoding the passed separator as a Base64 string.
454     String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
455     if (actualSeparator != null) {
456       conf.set(SEPARATOR_CONF_KEY,
457                Base64.encodeBytes(actualSeparator.getBytes()));
458     }
459 
460     // See if a non-default Mapper was set
461     String mapperClassName = conf.get(MAPPER_CONF_KEY);
462     Class mapperClass = mapperClassName != null ?
463         Class.forName(mapperClassName) : DEFAULT_MAPPER;
464 
465     String tableName = args[0];
466     Path inputDir = new Path(args[1]);
467     String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName);
468     Job job = new Job(conf, jobName);
469     job.setJarByClass(mapperClass);
470     FileInputFormat.setInputPaths(job, inputDir);
471     job.setInputFormatClass(TextInputFormat.class);
472     job.setMapperClass(mapperClass);
473     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
474     String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
475     if(StringUtils.isNotEmpty(conf.get(CREDENTIALS_LOCATION))) {
476       String fileLoc = conf.get(CREDENTIALS_LOCATION);
477       Credentials cred = Credentials.readTokenStorageFile(new Path(fileLoc), conf);
478       job.getCredentials().addAll(cred);
479     }
480 
481     if (hfileOutPath != null) {
482       if (!admin.tableExists(tableName)) {
483         String errorMsg = format("Table '%s' does not exist.", tableName);
484         if ("yes".equalsIgnoreCase(conf.get(CREATE_TABLE_CONF_KEY, "yes"))) {
485           LOG.warn(errorMsg);
486           // TODO: this is backwards. Instead of depending on the existence of a table,
487           // create a sane splits file for HFileOutputFormat based on data sampling.
488           createTable(admin, tableName, columns);
489         } else {
490           LOG.error(errorMsg);
491           throw new TableNotFoundException(errorMsg);
492         }
493       }
494       HTable table = new HTable(conf, tableName);
495       job.setReducerClass(PutSortReducer.class);
496       Path outputDir = new Path(hfileOutPath);
497       FileOutputFormat.setOutputPath(job, outputDir);
498       job.setMapOutputKeyClass(ImmutableBytesWritable.class);
499       if (mapperClass.equals(TsvImporterTextMapper.class)) {
500         job.setMapOutputValueClass(Text.class);
501         job.setReducerClass(TextSortReducer.class);
502       } else {
503         job.setMapOutputValueClass(Put.class);
504         job.setCombinerClass(PutCombiner.class);
505       }
506       HFileOutputFormat.configureIncrementalLoad(job, table);
507     } else {
508       if (!admin.tableExists(tableName)) {
509         String errorMsg = format("Table '%s' does not exist.", tableName);
510         LOG.error(errorMsg);
511         throw new TableNotFoundException(errorMsg);
512       }
513       if (mapperClass.equals(TsvImporterTextMapper.class)) {
514         usage(TsvImporterTextMapper.class.toString()
515             + " should not be used for non bulkloading case. use "
516             + TsvImporterMapper.class.toString()
517             + " or custom mapper whose value type is Put.");
518         System.exit(-1);
519       }
520       // No reducers. Just write straight to table. Call initTableReducerJob
521       // to set up the TableOutputFormat.
522       TableMapReduceUtil.initTableReducerJob(tableName, null, job);
523       job.setNumReduceTasks(0);
524     }
525 
526     TableMapReduceUtil.addDependencyJars(job);
527     TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
528         com.google.common.base.Function.class /* Guava used by TsvParser */);
529     return job;
530   }
531 
532   private static void createTable(HBaseAdmin admin, String tableName, String[] columns)
533       throws IOException {
534     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
535     Set<String> cfSet = new HashSet<String>();
536     for (String aColumn : columns) {
537       if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)
538           || TsvParser.TIMESTAMPKEY_COLUMN_SPEC.equals(aColumn)
539           || TsvParser.CELL_VISIBILITY_COLUMN_SPEC.equals(aColumn)
540           || TsvParser.CELL_TTL_COLUMN_SPEC.equals(aColumn)
541           || TsvParser.ATTRIBUTES_COLUMN_SPEC.equals(aColumn))
542         continue;
543       // we are only concerned with the first one (in case this is a cf:cq)
544       cfSet.add(aColumn.split(":", 2)[0]);
545     }
546     for (String cf : cfSet) {
547       HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(cf));
548       htd.addFamily(hcd);
549     }
550     LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.",
551       tableName, cfSet));
552     admin.createTable(htd);
553   }
554 
555   /*
556    * @param errorMsg Error message.  Can be null.
557    */
558   private static void usage(final String errorMsg) {
559     if (errorMsg != null && errorMsg.length() > 0) {
560       System.err.println("ERROR: " + errorMsg);
561     }
562     String usage =
563       "Usage: " + NAME + " -D"+ COLUMNS_CONF_KEY + "=a,b,c <tablename> <inputdir>\n" +
564       "\n" +
565       "Imports the given input directory of TSV data into the specified table.\n" +
566       "\n" +
567       "The column names of the TSV data must be specified using the -D" + COLUMNS_CONF_KEY + "\n" +
568       "option. This option takes the form of comma-separated column names, where each\n" +
569       "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
570       "column name " + TsvParser.ROWKEY_COLUMN_SPEC + " is used to designate that this column should be used\n" +
571       "as the row key for each imported record. You must specify exactly one column\n" +
572       "to be the row key, and you must specify a column name for every column that exists in the\n" +
573       "input data. Another special column" + TsvParser.TIMESTAMPKEY_COLUMN_SPEC +
574       " designates that this column should be\n" +
575       "used as timestamp for each record. Unlike " + TsvParser.ROWKEY_COLUMN_SPEC + ", " +
576       TsvParser.TIMESTAMPKEY_COLUMN_SPEC + " is optional.\n" +
577       "You must specify at most one column as timestamp key for each imported record.\n" +
578       "Record with invalid timestamps (blank, non-numeric) will be treated as bad record.\n" +
579       "Note: if you use this option, then '" + TIMESTAMP_CONF_KEY + "' option will be ignored.\n" +
580       "\n" +
581       TsvParser.ATTRIBUTES_COLUMN_SPEC+" can be used to specify Operation Attributes per record.\n"+
582       " Should be specified as key=>value where "+TsvParser.DEFAULT_ATTRIBUTES_COLUMN_INDEX+ " is used \n"+
583       " as the seperator.  Note that more than one OperationAttributes can be specified.\n"+
584       "By default importtsv will load data directly into HBase. To instead generate\n" +
585       "HFiles of data to prepare for a bulk data load, pass the option:\n" +
586       "  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
587       "  Note: if you do not use this option, then the target table must already exist in HBase\n" +
588       "\n" +
589       "Other options that may be specified with -D include:\n" +
590       "  -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
591       "  '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs\n" +
592       "  -D" + TIMESTAMP_CONF_KEY + "=currentTimeAsLong - use the specified timestamp for the import\n" +
593       "  -D" + MAPPER_CONF_KEY + "=my.Mapper - A user-defined Mapper to use instead of " +
594       DEFAULT_MAPPER.getName() + "\n" +
595       "  -D" + JOB_NAME_CONF_KEY + "=jobName - use the specified mapreduce job name for the import\n" +
596       "  -D" + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by this tool\n" +
597       "  Note: if you set this to 'no', then the target table must already exist in HBase\n" +
598       "\n" +
599       "For performance consider the following options:\n" +
600       "  -Dmapred.map.tasks.speculative.execution=false\n" +
601       "  -Dmapred.reduce.tasks.speculative.execution=false";
602 
603     System.err.println(usage);
604   }
605 
606   @Override
607   public int run(String[] args) throws Exception {
608     setConf(HBaseConfiguration.create(getConf()));
609     String[] otherArgs = new GenericOptionsParser(getConf(), args).getRemainingArgs();
610     if (otherArgs.length < 2) {
611       usage("Wrong number of arguments: " + otherArgs.length);
612       return -1;
613     }
614 
615     // When MAPPER_CONF_KEY is null, the user wants to use the provided TsvImporterMapper, so
616     // perform validation on these additional args. When it's not null, user has provided their
617     // own mapper, thus these validation are not relevant.
618     // TODO: validation for TsvImporterMapper, not this tool. Move elsewhere.
619     if (null == getConf().get(MAPPER_CONF_KEY)) {
620       // Make sure columns are specified
621       String columns[] = getConf().getStrings(COLUMNS_CONF_KEY);
622       if (columns == null) {
623         usage("No columns specified. Please specify with -D" +
624             COLUMNS_CONF_KEY+"=...");
625         return -1;
626       }
627 
628       // Make sure they specify exactly one column as the row key
629       int rowkeysFound = 0;
630       for (String col : columns) {
631         if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
632       }
633       if (rowkeysFound != 1) {
634         usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
635         return -1;
636       }
637 
638       // Make sure we have at most one column as the timestamp key
639       int tskeysFound = 0;
640       for (String col : columns) {
641         if (col.equals(TsvParser.TIMESTAMPKEY_COLUMN_SPEC))
642           tskeysFound++;
643       }
644       if (tskeysFound > 1) {
645         usage("Must specify at most one column as "
646             + TsvParser.TIMESTAMPKEY_COLUMN_SPEC);
647         return -1;
648       }
649 
650       int attrKeysFound = 0;
651       for (String col : columns) {
652         if (col.equals(TsvParser.ATTRIBUTES_COLUMN_SPEC))
653           attrKeysFound++;
654       }
655       if (attrKeysFound > 1) {
656         usage("Must specify at most one column as "
657             + TsvParser.ATTRIBUTES_COLUMN_SPEC);
658         return -1;
659       }
660 
661       // Make sure one or more columns are specified excluding rowkey and
662       // timestamp key
663       if (columns.length - (rowkeysFound + tskeysFound + attrKeysFound) < 1) {
664         usage("One or more columns in addition to the row key and timestamp(optional) are required");
665         return -1;
666       }
667     }
668 
669     // If timestamp option is not specified, use current system time.
670     long timstamp = getConf().getLong(TIMESTAMP_CONF_KEY, System.currentTimeMillis());
671 
672     // Set it back to replace invalid timestamp (non-numeric) with current
673     // system time
674     getConf().setLong(TIMESTAMP_CONF_KEY, timstamp);
675 
676     Job job = createSubmittableJob(getConf(), otherArgs);
677     return job.waitForCompletion(true) ? 0 : 1;
678   }
679 
680   public static void main(String[] args) throws Exception {
681     int status = ToolRunner.run(new ImportTsv(), args);
682     System.exit(status);
683   }
684 }