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