001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.mapreduce;
020
021import static java.lang.String.format;
022
023import java.io.File;
024import java.io.IOException;
025import java.util.ArrayList;
026import java.util.Base64;
027import java.util.HashSet;
028import java.util.Set;
029
030import org.apache.commons.lang3.StringUtils;
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.conf.Configured;
033import org.apache.hadoop.fs.Path;
034import org.apache.hadoop.hbase.HBaseConfiguration;
035import org.apache.hadoop.hbase.HColumnDescriptor;
036import org.apache.hadoop.hbase.HConstants;
037import org.apache.hadoop.hbase.HTableDescriptor;
038import org.apache.hadoop.hbase.TableName;
039import org.apache.hadoop.hbase.TableNotEnabledException;
040import org.apache.hadoop.hbase.TableNotFoundException;
041import org.apache.yetus.audience.InterfaceAudience;
042import org.slf4j.Logger;
043import org.slf4j.LoggerFactory;
044import org.apache.hadoop.hbase.client.Admin;
045import org.apache.hadoop.hbase.client.Connection;
046import org.apache.hadoop.hbase.client.ConnectionFactory;
047import org.apache.hadoop.hbase.client.Put;
048import org.apache.hadoop.hbase.client.RegionLocator;
049import org.apache.hadoop.hbase.client.Table;
050import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
051import org.apache.hadoop.hbase.util.Bytes;
052import org.apache.hadoop.hbase.util.Pair;
053import org.apache.hadoop.io.Text;
054import org.apache.hadoop.mapreduce.Job;
055import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
056import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
057import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
058import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
059import org.apache.hadoop.security.Credentials;
060import org.apache.hadoop.util.Tool;
061import org.apache.hadoop.util.ToolRunner;
062
063import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
064import org.apache.hadoop.hbase.client.TableDescriptor;
065import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
066import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
067import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
068
069/**
070 * Tool to import data from a TSV file.
071 *
072 * This tool is rather simplistic - it doesn't do any quoting or
073 * escaping, but is useful for many data loads.
074 *
075 * @see ImportTsv#usage(String)
076 */
077@InterfaceAudience.Public
078public class ImportTsv extends Configured implements Tool {
079
080  protected static final Logger LOG = LoggerFactory.getLogger(ImportTsv.class);
081
082  final static String NAME = "importtsv";
083
084  public final static String MAPPER_CONF_KEY = "importtsv.mapper.class";
085  public final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
086  public final static String TIMESTAMP_CONF_KEY = "importtsv.timestamp";
087  public final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
088  // TODO: the rest of these configs are used exclusively by TsvImporterMapper.
089  // Move them out of the tool and let the mapper handle its own validation.
090  public final static String DRY_RUN_CONF_KEY = "importtsv.dry.run";
091  // If true, bad lines are logged to stderr. Default: false.
092  public final static String LOG_BAD_LINES_CONF_KEY = "importtsv.log.bad.lines";
093  public final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
094  public final static String SKIP_EMPTY_COLUMNS = "importtsv.skip.empty.columns";
095  public final static String COLUMNS_CONF_KEY = "importtsv.columns";
096  public final static String SEPARATOR_CONF_KEY = "importtsv.separator";
097  public final static String ATTRIBUTE_SEPERATOR_CONF_KEY = "attributes.seperator";
098  //This config is used to propagate credentials from parent MR jobs which launch
099  //ImportTSV jobs. SEE IntegrationTestImportTsv.
100  public final static String CREDENTIALS_LOCATION = "credentials_location";
101  final static String DEFAULT_SEPARATOR = "\t";
102  final static String DEFAULT_ATTRIBUTES_SEPERATOR = "=>";
103  final static String DEFAULT_MULTIPLE_ATTRIBUTES_SEPERATOR = ",";
104  final static Class DEFAULT_MAPPER = TsvImporterMapper.class;
105  public final static String CREATE_TABLE_CONF_KEY = "create.table";
106  public final static String NO_STRICT_COL_FAMILY = "no.strict";
107  /**
108   * If table didn't exist and was created in dry-run mode, this flag is
109   * flipped to delete it when MR ends.
110   */
111  private static boolean DRY_RUN_TABLE_CREATED;
112
113  public static class TsvParser {
114    /**
115     * Column families and qualifiers mapped to the TSV columns
116     */
117    private final byte[][] families;
118    private final byte[][] qualifiers;
119
120    private final byte separatorByte;
121
122    private int rowKeyColumnIndex;
123
124    private int maxColumnCount;
125
126    // Default value must be negative
127    public static final int DEFAULT_TIMESTAMP_COLUMN_INDEX = -1;
128
129    private int timestampKeyColumnIndex = DEFAULT_TIMESTAMP_COLUMN_INDEX;
130
131    public static final String ROWKEY_COLUMN_SPEC = "HBASE_ROW_KEY";
132
133    public static final String TIMESTAMPKEY_COLUMN_SPEC = "HBASE_TS_KEY";
134
135    public static final String ATTRIBUTES_COLUMN_SPEC = "HBASE_ATTRIBUTES_KEY";
136
137    public static final String CELL_VISIBILITY_COLUMN_SPEC = "HBASE_CELL_VISIBILITY";
138
139    public static final String CELL_TTL_COLUMN_SPEC = "HBASE_CELL_TTL";
140
141    private int attrKeyColumnIndex = DEFAULT_ATTRIBUTES_COLUMN_INDEX;
142
143    public static final int DEFAULT_ATTRIBUTES_COLUMN_INDEX = -1;
144
145    public static final int DEFAULT_CELL_VISIBILITY_COLUMN_INDEX = -1;
146
147    public static final int DEFAULT_CELL_TTL_COLUMN_INDEX = -1;
148
149    private int cellVisibilityColumnIndex = DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
150
151    private int cellTTLColumnIndex = DEFAULT_CELL_TTL_COLUMN_INDEX;
152
153    /**
154     * @param columnsSpecification the list of columns to parser out, comma separated.
155     * The row key should be the special token TsvParser.ROWKEY_COLUMN_SPEC
156     * @param separatorStr
157     */
158    public TsvParser(String columnsSpecification, String separatorStr) {
159      // Configure separator
160      byte[] separator = Bytes.toBytes(separatorStr);
161      Preconditions.checkArgument(separator.length == 1,
162        "TsvParser only supports single-byte separators");
163      separatorByte = separator[0];
164
165      // Configure columns
166      ArrayList<String> columnStrings = Lists.newArrayList(
167        Splitter.on(',').trimResults().split(columnsSpecification));
168
169      maxColumnCount = columnStrings.size();
170      families = new byte[maxColumnCount][];
171      qualifiers = new byte[maxColumnCount][];
172
173      for (int i = 0; i < columnStrings.size(); i++) {
174        String str = columnStrings.get(i);
175        if (ROWKEY_COLUMN_SPEC.equals(str)) {
176          rowKeyColumnIndex = i;
177          continue;
178        }
179        if (TIMESTAMPKEY_COLUMN_SPEC.equals(str)) {
180          timestampKeyColumnIndex = i;
181          continue;
182        }
183        if (ATTRIBUTES_COLUMN_SPEC.equals(str)) {
184          attrKeyColumnIndex = i;
185          continue;
186        }
187        if (CELL_VISIBILITY_COLUMN_SPEC.equals(str)) {
188          cellVisibilityColumnIndex = i;
189          continue;
190        }
191        if (CELL_TTL_COLUMN_SPEC.equals(str)) {
192          cellTTLColumnIndex = i;
193          continue;
194        }
195        String[] parts = str.split(":", 2);
196        if (parts.length == 1) {
197          families[i] = str.getBytes();
198          qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
199        } else {
200          families[i] = parts[0].getBytes();
201          qualifiers[i] = parts[1].getBytes();
202        }
203      }
204    }
205
206    public boolean hasTimestamp() {
207      return timestampKeyColumnIndex != DEFAULT_TIMESTAMP_COLUMN_INDEX;
208    }
209
210    public int getTimestampKeyColumnIndex() {
211      return timestampKeyColumnIndex;
212    }
213
214    public boolean hasAttributes() {
215      return attrKeyColumnIndex != DEFAULT_ATTRIBUTES_COLUMN_INDEX;
216    }
217
218    public boolean hasCellVisibility() {
219      return cellVisibilityColumnIndex != DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
220    }
221
222    public boolean hasCellTTL() {
223      return cellTTLColumnIndex != DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
224    }
225
226    public int getAttributesKeyColumnIndex() {
227      return attrKeyColumnIndex;
228    }
229
230    public int getCellVisibilityColumnIndex() {
231      return cellVisibilityColumnIndex;
232    }
233
234    public int getCellTTLColumnIndex() {
235      return cellTTLColumnIndex;
236    }
237
238    public int getRowKeyColumnIndex() {
239      return rowKeyColumnIndex;
240    }
241
242    public byte[] getFamily(int idx) {
243      return families[idx];
244    }
245    public byte[] getQualifier(int idx) {
246      return qualifiers[idx];
247    }
248
249    public ParsedLine parse(byte[] lineBytes, int length)
250    throws BadTsvLineException {
251      // Enumerate separator offsets
252      ArrayList<Integer> tabOffsets = new ArrayList<>(maxColumnCount);
253      for (int i = 0; i < length; i++) {
254        if (lineBytes[i] == separatorByte) {
255          tabOffsets.add(i);
256        }
257      }
258      if (tabOffsets.isEmpty()) {
259        throw new BadTsvLineException("No delimiter");
260      }
261
262      tabOffsets.add(length);
263
264      if (tabOffsets.size() > maxColumnCount) {
265        throw new BadTsvLineException("Excessive columns");
266      } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
267        throw new BadTsvLineException("No row key");
268      } else if (hasTimestamp()
269          && tabOffsets.size() <= getTimestampKeyColumnIndex()) {
270        throw new BadTsvLineException("No timestamp");
271      } else if (hasAttributes() && tabOffsets.size() <= getAttributesKeyColumnIndex()) {
272        throw new BadTsvLineException("No attributes specified");
273      } else if (hasCellVisibility() && tabOffsets.size() <= getCellVisibilityColumnIndex()) {
274        throw new BadTsvLineException("No cell visibility specified");
275      } else if (hasCellTTL() && tabOffsets.size() <= getCellTTLColumnIndex()) {
276        throw new BadTsvLineException("No cell TTL specified");
277      }
278      return new ParsedLine(tabOffsets, lineBytes);
279    }
280
281    class ParsedLine {
282      private final ArrayList<Integer> tabOffsets;
283      private byte[] lineBytes;
284
285      ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
286        this.tabOffsets = tabOffsets;
287        this.lineBytes = lineBytes;
288      }
289
290      public int getRowKeyOffset() {
291        return getColumnOffset(rowKeyColumnIndex);
292      }
293      public int getRowKeyLength() {
294        return getColumnLength(rowKeyColumnIndex);
295      }
296
297      public long getTimestamp(long ts) throws BadTsvLineException {
298        // Return ts if HBASE_TS_KEY is not configured in column spec
299        if (!hasTimestamp()) {
300          return ts;
301        }
302
303        String timeStampStr = Bytes.toString(lineBytes,
304            getColumnOffset(timestampKeyColumnIndex),
305            getColumnLength(timestampKeyColumnIndex));
306        try {
307          return Long.parseLong(timeStampStr);
308        } catch (NumberFormatException nfe) {
309          // treat this record as bad record
310          throw new BadTsvLineException("Invalid timestamp " + timeStampStr);
311        }
312      }
313
314      private String getAttributes() {
315        if (!hasAttributes()) {
316          return null;
317        } else {
318          return Bytes.toString(lineBytes, getColumnOffset(attrKeyColumnIndex),
319              getColumnLength(attrKeyColumnIndex));
320        }
321      }
322
323      public String[] getIndividualAttributes() {
324        String attributes = getAttributes();
325        if (attributes != null) {
326          return attributes.split(DEFAULT_MULTIPLE_ATTRIBUTES_SEPERATOR);
327        } else {
328          return null;
329        }
330      }
331
332      public int getAttributeKeyOffset() {
333        if (hasAttributes()) {
334          return getColumnOffset(attrKeyColumnIndex);
335        } else {
336          return DEFAULT_ATTRIBUTES_COLUMN_INDEX;
337        }
338      }
339
340      public int getAttributeKeyLength() {
341        if (hasAttributes()) {
342          return getColumnLength(attrKeyColumnIndex);
343        } else {
344          return DEFAULT_ATTRIBUTES_COLUMN_INDEX;
345        }
346      }
347
348      public int getCellVisibilityColumnOffset() {
349        if (hasCellVisibility()) {
350          return getColumnOffset(cellVisibilityColumnIndex);
351        } else {
352          return DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
353        }
354      }
355
356      public int getCellVisibilityColumnLength() {
357        if (hasCellVisibility()) {
358          return getColumnLength(cellVisibilityColumnIndex);
359        } else {
360          return DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
361        }
362      }
363
364      public String getCellVisibility() {
365        if (!hasCellVisibility()) {
366          return null;
367        } else {
368          return Bytes.toString(lineBytes, getColumnOffset(cellVisibilityColumnIndex),
369              getColumnLength(cellVisibilityColumnIndex));
370        }
371      }
372
373      public int getCellTTLColumnOffset() {
374        if (hasCellTTL()) {
375          return getColumnOffset(cellTTLColumnIndex);
376        } else {
377          return DEFAULT_CELL_TTL_COLUMN_INDEX;
378        }
379      }
380
381      public int getCellTTLColumnLength() {
382        if (hasCellTTL()) {
383          return getColumnLength(cellTTLColumnIndex);
384        } else {
385          return DEFAULT_CELL_TTL_COLUMN_INDEX;
386        }
387      }
388
389      public long getCellTTL() {
390        if (!hasCellTTL()) {
391          return 0;
392        } else {
393          return Bytes.toLong(lineBytes, getColumnOffset(cellTTLColumnIndex),
394              getColumnLength(cellTTLColumnIndex));
395        }
396      }
397
398      public int getColumnOffset(int idx) {
399        if (idx > 0)
400          return tabOffsets.get(idx - 1) + 1;
401        else
402          return 0;
403      }
404      public int getColumnLength(int idx) {
405        return tabOffsets.get(idx) - getColumnOffset(idx);
406      }
407      public int getColumnCount() {
408        return tabOffsets.size();
409      }
410      public byte[] getLineBytes() {
411        return lineBytes;
412      }
413    }
414
415    public static class BadTsvLineException extends Exception {
416      public BadTsvLineException(String err) {
417        super(err);
418      }
419      private static final long serialVersionUID = 1L;
420    }
421
422    /**
423     * Return starting position and length of row key from the specified line bytes.
424     * @param lineBytes
425     * @param length
426     * @return Pair of row key offset and length.
427     * @throws BadTsvLineException
428     */
429    public Pair<Integer, Integer> parseRowKey(byte[] lineBytes, int length)
430        throws BadTsvLineException {
431      int rkColumnIndex = 0;
432      int startPos = 0, endPos = 0;
433      for (int i = 0; i <= length; i++) {
434        if (i == length || lineBytes[i] == separatorByte) {
435          endPos = i - 1;
436          if (rkColumnIndex++ == getRowKeyColumnIndex()) {
437            if ((endPos + 1) == startPos) {
438              throw new BadTsvLineException("Empty value for ROW KEY.");
439            }
440            break;
441          } else {
442            startPos = endPos + 2;
443          }
444        }
445        if (i == length) {
446          throw new BadTsvLineException(
447              "Row key does not exist as number of columns in the line"
448                  + " are less than row key position.");
449        }
450      }
451      return new Pair<>(startPos, endPos - startPos + 1);
452    }
453  }
454
455  /**
456   * Sets up the actual job.
457   *
458   * @param conf  The current configuration.
459   * @param args  The command line parameters.
460   * @return The newly created job.
461   * @throws IOException When setting up the job fails.
462   */
463  protected static Job createSubmittableJob(Configuration conf, String[] args)
464      throws IOException, ClassNotFoundException {
465    Job job = null;
466    boolean isDryRun = conf.getBoolean(DRY_RUN_CONF_KEY, false);
467    try (Connection connection = ConnectionFactory.createConnection(conf)) {
468      try (Admin admin = connection.getAdmin()) {
469        // Support non-XML supported characters
470        // by re-encoding the passed separator as a Base64 string.
471        String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
472        if (actualSeparator != null) {
473          conf.set(SEPARATOR_CONF_KEY,
474              Bytes.toString(Base64.getEncoder().encode(actualSeparator.getBytes())));
475        }
476
477        // See if a non-default Mapper was set
478        String mapperClassName = conf.get(MAPPER_CONF_KEY);
479        Class mapperClass = mapperClassName != null? Class.forName(mapperClassName): DEFAULT_MAPPER;
480
481        TableName tableName = TableName.valueOf(args[0]);
482        Path inputDir = new Path(args[1]);
483        String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName.getNameAsString());
484        job = Job.getInstance(conf, jobName);
485        job.setJarByClass(mapperClass);
486        FileInputFormat.setInputPaths(job, inputDir);
487        job.setInputFormatClass(TextInputFormat.class);
488        job.setMapperClass(mapperClass);
489        job.setMapOutputKeyClass(ImmutableBytesWritable.class);
490        String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
491        String[] columns = conf.getStrings(COLUMNS_CONF_KEY);
492        if(StringUtils.isNotEmpty(conf.get(CREDENTIALS_LOCATION))) {
493          String fileLoc = conf.get(CREDENTIALS_LOCATION);
494          Credentials cred = Credentials.readTokenStorageFile(new File(fileLoc), conf);
495          job.getCredentials().addAll(cred);
496        }
497
498        if (hfileOutPath != null) {
499          if (!admin.tableExists(tableName)) {
500            LOG.warn(format("Table '%s' does not exist.", tableName));
501            if ("yes".equalsIgnoreCase(conf.get(CREATE_TABLE_CONF_KEY, "yes"))) {
502              // TODO: this is backwards. Instead of depending on the existence of a table,
503              // create a sane splits file for HFileOutputFormat based on data sampling.
504              createTable(admin, tableName, columns);
505              if (isDryRun) {
506                LOG.warn("Dry run: Table will be deleted at end of dry run.");
507                synchronized (ImportTsv.class) {
508                  DRY_RUN_TABLE_CREATED = true;
509                }
510              }
511            } else {
512              String errorMsg =
513                  format("Table '%s' does not exist and '%s' is set to no.", tableName,
514                      CREATE_TABLE_CONF_KEY);
515              LOG.error(errorMsg);
516              throw new TableNotFoundException(errorMsg);
517            }
518          }
519          try (Table table = connection.getTable(tableName);
520              RegionLocator regionLocator = connection.getRegionLocator(tableName)) {
521            boolean noStrict = conf.getBoolean(NO_STRICT_COL_FAMILY, false);
522            // if no.strict is false then check column family
523            if(!noStrict) {
524              ArrayList<String> unmatchedFamilies = new ArrayList<>();
525              Set<String> cfSet = getColumnFamilies(columns);
526              TableDescriptor tDesc = table.getDescriptor();
527              for (String cf : cfSet) {
528                if(!tDesc.hasColumnFamily(Bytes.toBytes(cf))) {
529                  unmatchedFamilies.add(cf);
530                }
531              }
532              if(unmatchedFamilies.size() > 0) {
533                ArrayList<String> familyNames = new ArrayList<>();
534                for (ColumnFamilyDescriptor family : table.getDescriptor().getColumnFamilies()) {
535                  familyNames.add(family.getNameAsString());
536                }
537                String msg =
538                    "Column Families " + unmatchedFamilies + " specified in " + COLUMNS_CONF_KEY
539                    + " does not match with any of the table " + tableName
540                    + " column families " + familyNames + ".\n"
541                    + "To disable column family check, use -D" + NO_STRICT_COL_FAMILY
542                    + "=true.\n";
543                usage(msg);
544                System.exit(-1);
545              }
546            }
547            if (mapperClass.equals(TsvImporterTextMapper.class)) {
548              job.setMapOutputValueClass(Text.class);
549              job.setReducerClass(TextSortReducer.class);
550            } else {
551              job.setMapOutputValueClass(Put.class);
552              job.setCombinerClass(PutCombiner.class);
553              job.setReducerClass(PutSortReducer.class);
554            }
555            if (!isDryRun) {
556              Path outputDir = new Path(hfileOutPath);
557              FileOutputFormat.setOutputPath(job, outputDir);
558              HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(),
559                  regionLocator);
560            }
561          }
562        } else {
563          if (!admin.tableExists(tableName)) {
564            String errorMsg = format("Table '%s' does not exist.", tableName);
565            LOG.error(errorMsg);
566            throw new TableNotFoundException(errorMsg);
567          }
568          if (mapperClass.equals(TsvImporterTextMapper.class)) {
569            usage(TsvImporterTextMapper.class.toString()
570                + " should not be used for non bulkloading case. use "
571                + TsvImporterMapper.class.toString()
572                + " or custom mapper whose value type is Put.");
573            System.exit(-1);
574          }
575          if (!isDryRun) {
576            // No reducers. Just write straight to table. Call initTableReducerJob
577            // to set up the TableOutputFormat.
578            TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job);
579          }
580          job.setNumReduceTasks(0);
581        }
582        if (isDryRun) {
583          job.setOutputFormatClass(NullOutputFormat.class);
584          job.getConfiguration().setStrings("io.serializations",
585              job.getConfiguration().get("io.serializations"),
586              MutationSerialization.class.getName(), ResultSerialization.class.getName(),
587              CellSerialization.class.getName());
588        }
589        TableMapReduceUtil.addDependencyJars(job);
590        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
591            org.apache.hbase.thirdparty.com.google.common.base.Function.class /* Guava used by TsvParser */);
592      }
593    }
594    return job;
595  }
596
597  private static void createTable(Admin admin, TableName tableName, String[] columns)
598      throws IOException {
599    HTableDescriptor htd = new HTableDescriptor(tableName);
600    Set<String> cfSet = getColumnFamilies(columns);
601    for (String cf : cfSet) {
602      HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(cf));
603      htd.addFamily(hcd);
604    }
605    LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.",
606      tableName, cfSet));
607    admin.createTable(htd);
608  }
609
610  private static void deleteTable(Configuration conf, String[] args) {
611    TableName tableName = TableName.valueOf(args[0]);
612    try (Connection connection = ConnectionFactory.createConnection(conf);
613         Admin admin = connection.getAdmin()) {
614      try {
615        admin.disableTable(tableName);
616      } catch (TableNotEnabledException e) {
617        LOG.debug("Dry mode: Table: " + tableName + " already disabled, so just deleting it.");
618      }
619      admin.deleteTable(tableName);
620    } catch (IOException e) {
621      LOG.error(format("***Dry run: Failed to delete table '%s'.***%n%s", tableName,
622          e.toString()));
623      return;
624    }
625    LOG.info(format("Dry run: Deleted table '%s'.", tableName));
626  }
627
628  private static Set<String> getColumnFamilies(String[] columns) {
629    Set<String> cfSet = new HashSet<>();
630    for (String aColumn : columns) {
631      if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)
632          || TsvParser.TIMESTAMPKEY_COLUMN_SPEC.equals(aColumn)
633          || TsvParser.CELL_VISIBILITY_COLUMN_SPEC.equals(aColumn)
634          || TsvParser.CELL_TTL_COLUMN_SPEC.equals(aColumn)
635          || TsvParser.ATTRIBUTES_COLUMN_SPEC.equals(aColumn))
636        continue;
637      // we are only concerned with the first one (in case this is a cf:cq)
638      cfSet.add(aColumn.split(":", 2)[0]);
639    }
640    return cfSet;
641  }
642
643  /*
644   * @param errorMsg Error message.  Can be null.
645   */
646  private static void usage(final String errorMsg) {
647    if (errorMsg != null && errorMsg.length() > 0) {
648      System.err.println("ERROR: " + errorMsg);
649    }
650    String usage =
651      "Usage: " + NAME + " -D"+ COLUMNS_CONF_KEY + "=a,b,c <tablename> <inputdir>\n" +
652      "\n" +
653      "Imports the given input directory of TSV data into the specified table.\n" +
654      "\n" +
655      "The column names of the TSV data must be specified using the -D" + COLUMNS_CONF_KEY + "\n" +
656      "option. This option takes the form of comma-separated column names, where each\n" +
657      "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
658      "column name " + TsvParser.ROWKEY_COLUMN_SPEC + " is used to designate that this column should be used\n" +
659      "as the row key for each imported record. You must specify exactly one column\n" +
660      "to be the row key, and you must specify a column name for every column that exists in the\n" +
661      "input data. Another special column" + TsvParser.TIMESTAMPKEY_COLUMN_SPEC +
662      " designates that this column should be\n" +
663      "used as timestamp for each record. Unlike " + TsvParser.ROWKEY_COLUMN_SPEC + ", " +
664      TsvParser.TIMESTAMPKEY_COLUMN_SPEC + " is optional." + "\n" +
665      "You must specify at most one column as timestamp key for each imported record.\n" +
666      "Record with invalid timestamps (blank, non-numeric) will be treated as bad record.\n" +
667      "Note: if you use this option, then '" + TIMESTAMP_CONF_KEY + "' option will be ignored.\n" +
668      "\n" +
669      "Other special columns that can be specified are " + TsvParser.CELL_TTL_COLUMN_SPEC +
670      " and " + TsvParser.CELL_VISIBILITY_COLUMN_SPEC + ".\n" +
671      TsvParser.CELL_TTL_COLUMN_SPEC + " designates that this column will be used " +
672      "as a Cell's Time To Live (TTL) attribute.\n" +
673      TsvParser.CELL_VISIBILITY_COLUMN_SPEC + " designates that this column contains the " +
674      "visibility label expression.\n" +
675      "\n" +
676      TsvParser.ATTRIBUTES_COLUMN_SPEC+" can be used to specify Operation Attributes per record.\n"+
677      " Should be specified as key=>value where "+TsvParser.DEFAULT_ATTRIBUTES_COLUMN_INDEX+ " is used \n"+
678      " as the seperator.  Note that more than one OperationAttributes can be specified.\n"+
679      "By default importtsv will load data directly into HBase. To instead generate\n" +
680      "HFiles of data to prepare for a bulk data load, pass the option:\n" +
681      "  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
682      "  Note: if you do not use this option, then the target table must already exist in HBase\n" +
683      "\n" +
684      "Other options that may be specified with -D include:\n" +
685      "  -D" + DRY_RUN_CONF_KEY + "=true - Dry run mode. Data is not actually populated into" +
686      " table. If table does not exist, it is created but deleted in the end.\n" +
687      "  -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
688      "  -D" + LOG_BAD_LINES_CONF_KEY + "=true - logs invalid lines to stderr\n" +
689      "  -D" + SKIP_EMPTY_COLUMNS + "=false - If true then skip empty columns in bulk import\n" +
690      "  '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs\n" +
691      "  -D" + TIMESTAMP_CONF_KEY + "=currentTimeAsLong - use the specified timestamp for the import\n" +
692      "  -D" + MAPPER_CONF_KEY + "=my.Mapper - A user-defined Mapper to use instead of " +
693      DEFAULT_MAPPER.getName() + "\n" +
694      "  -D" + JOB_NAME_CONF_KEY + "=jobName - use the specified mapreduce job name for the import\n" +
695      "  -D" + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by this tool\n" +
696      "  Note: if you set this to 'no', then the target table must already exist in HBase\n" +
697      "  -D" + NO_STRICT_COL_FAMILY + "=true - ignore column family check in hbase table. " +
698      "Default is false\n\n" +
699      "For performance consider the following options:\n" +
700      "  -Dmapreduce.map.speculative=false\n" +
701      "  -Dmapreduce.reduce.speculative=false";
702
703    System.err.println(usage);
704  }
705
706  @Override
707  public int run(String[] args) throws Exception {
708    if (args.length < 2) {
709      usage("Wrong number of arguments: " + args.length);
710      return -1;
711    }
712
713    // When MAPPER_CONF_KEY is null, the user wants to use the provided TsvImporterMapper, so
714    // perform validation on these additional args. When it's not null, user has provided their
715    // own mapper, thus these validation are not relevant.
716    // TODO: validation for TsvImporterMapper, not this tool. Move elsewhere.
717    if (null == getConf().get(MAPPER_CONF_KEY)) {
718      // Make sure columns are specified
719      String[] columns = getConf().getStrings(COLUMNS_CONF_KEY);
720      if (columns == null) {
721        usage("No columns specified. Please specify with -D" +
722            COLUMNS_CONF_KEY+"=...");
723        return -1;
724      }
725
726      // Make sure they specify exactly one column as the row key
727      int rowkeysFound = 0;
728      for (String col : columns) {
729        if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
730      }
731      if (rowkeysFound != 1) {
732        usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
733        return -1;
734      }
735
736      // Make sure we have at most one column as the timestamp key
737      int tskeysFound = 0;
738      for (String col : columns) {
739        if (col.equals(TsvParser.TIMESTAMPKEY_COLUMN_SPEC))
740          tskeysFound++;
741      }
742      if (tskeysFound > 1) {
743        usage("Must specify at most one column as "
744            + TsvParser.TIMESTAMPKEY_COLUMN_SPEC);
745        return -1;
746      }
747
748      int attrKeysFound = 0;
749      for (String col : columns) {
750        if (col.equals(TsvParser.ATTRIBUTES_COLUMN_SPEC))
751          attrKeysFound++;
752      }
753      if (attrKeysFound > 1) {
754        usage("Must specify at most one column as "
755            + TsvParser.ATTRIBUTES_COLUMN_SPEC);
756        return -1;
757      }
758
759      // Make sure one or more columns are specified excluding rowkey and
760      // timestamp key
761      if (columns.length - (rowkeysFound + tskeysFound + attrKeysFound) < 1) {
762        usage("One or more columns in addition to the row key and timestamp(optional) are required");
763        return -1;
764      }
765    }
766
767    // If timestamp option is not specified, use current system time.
768    long timstamp = getConf().getLong(TIMESTAMP_CONF_KEY, System.currentTimeMillis());
769
770    // Set it back to replace invalid timestamp (non-numeric) with current
771    // system time
772    getConf().setLong(TIMESTAMP_CONF_KEY, timstamp);
773
774    synchronized (ImportTsv.class) {
775      DRY_RUN_TABLE_CREATED = false;
776    }
777    Job job = createSubmittableJob(getConf(), args);
778    boolean success = job.waitForCompletion(true);
779    boolean delete = false;
780    synchronized (ImportTsv.class) {
781      delete = DRY_RUN_TABLE_CREATED;
782    }
783    if (delete) {
784      deleteTable(getConf(), args);
785    }
786    return success ? 0 : 1;
787  }
788
789  public static void main(String[] args) throws Exception {
790    int status = ToolRunner.run(HBaseConfiguration.create(), new ImportTsv(), args);
791    System.exit(status);
792  }
793}