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