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