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 java.io.ByteArrayInputStream;
022import java.io.DataInput;
023import java.io.DataInputStream;
024import java.io.DataOutput;
025import java.io.IOException;
026import java.lang.reflect.InvocationTargetException;
027import java.lang.reflect.Method;
028import java.util.ArrayList;
029import java.util.Collections;
030import java.util.List;
031import java.util.Locale;
032import java.util.Map;
033import java.util.TreeMap;
034import java.util.UUID;
035
036import org.apache.hadoop.conf.Configuration;
037import org.apache.hadoop.conf.Configured;
038import org.apache.hadoop.fs.FileSystem;
039import org.apache.hadoop.fs.Path;
040import org.apache.hadoop.hbase.Cell;
041import org.apache.hadoop.hbase.CellComparator;
042import org.apache.hadoop.hbase.CellUtil;
043import org.apache.hadoop.hbase.HBaseConfiguration;
044import org.apache.hadoop.hbase.PrivateCellUtil;
045import org.apache.hadoop.hbase.KeyValue;
046import org.apache.hadoop.hbase.KeyValueUtil;
047import org.apache.hadoop.hbase.TableName;
048import org.apache.hadoop.hbase.Tag;
049import org.apache.hadoop.hbase.ZooKeeperConnectionException;
050import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
051import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
052import org.apache.yetus.audience.InterfaceAudience;
053import org.apache.hadoop.hbase.client.Admin;
054import org.apache.hadoop.hbase.client.Connection;
055import org.apache.hadoop.hbase.client.ConnectionFactory;
056import org.apache.hadoop.hbase.client.Delete;
057import org.apache.hadoop.hbase.client.Durability;
058import org.apache.hadoop.hbase.client.Mutation;
059import org.apache.hadoop.hbase.client.Put;
060import org.apache.hadoop.hbase.client.RegionLocator;
061import org.apache.hadoop.hbase.client.Result;
062import org.apache.hadoop.hbase.client.Table;
063import org.apache.hadoop.hbase.filter.Filter;
064import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
065import org.apache.hadoop.hbase.util.Bytes;
066import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
067import org.apache.hadoop.io.RawComparator;
068import org.apache.hadoop.io.WritableComparable;
069import org.apache.hadoop.io.WritableComparator;
070import org.apache.hadoop.mapreduce.Job;
071import org.apache.hadoop.mapreduce.Partitioner;
072import org.apache.hadoop.mapreduce.Reducer;
073import org.apache.hadoop.mapreduce.TaskCounter;
074import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
075import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
076import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
077import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
078import org.apache.hadoop.util.Tool;
079import org.apache.hadoop.util.ToolRunner;
080import org.apache.zookeeper.KeeperException;
081import org.slf4j.Logger;
082import org.slf4j.LoggerFactory;
083
084
085/**
086 * Import data written by {@link Export}.
087 */
088@InterfaceAudience.Public
089public class Import extends Configured implements Tool {
090  private static final Logger LOG = LoggerFactory.getLogger(Import.class);
091  final static String NAME = "import";
092  public final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
093  public final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
094  public final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
095  public final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
096  public final static String TABLE_NAME = "import.table.name";
097  public final static String WAL_DURABILITY = "import.wal.durability";
098  public final static String HAS_LARGE_RESULT= "import.bulk.hasLargeResult";
099
100  private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
101
102  public static class CellWritableComparablePartitioner
103      extends Partitioner<CellWritableComparable, Cell> {
104    private static CellWritableComparable[] START_KEYS = null;
105    @Override
106    public int getPartition(CellWritableComparable key, Cell value,
107        int numPartitions) {
108      for (int i = 0; i < START_KEYS.length; ++i) {
109        if (key.compareTo(START_KEYS[i]) <= 0) {
110          return i;
111        }
112      }
113      return START_KEYS.length;
114    }
115
116  }
117
118  public static class CellWritableComparable
119      implements WritableComparable<CellWritableComparable> {
120
121    private Cell kv = null;
122
123    static {
124      // register this comparator
125      WritableComparator.define(CellWritableComparable.class,
126          new CellWritableComparator());
127    }
128
129    public CellWritableComparable() {
130    }
131
132    public CellWritableComparable(Cell kv) {
133      this.kv = kv;
134    }
135
136    @Override
137    public void write(DataOutput out) throws IOException {
138      out.writeInt(PrivateCellUtil.estimatedSerializedSizeOfKey(kv));
139      out.writeInt(0);
140      PrivateCellUtil.writeFlatKey(kv, out);
141    }
142
143    @Override
144    public void readFields(DataInput in) throws IOException {
145      kv = KeyValue.create(in);
146    }
147
148    @Override
149    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS",
150        justification = "This is wrong, yes, but we should be purging Writables, not fixing them")
151    public int compareTo(CellWritableComparable o) {
152      return CellComparator.getInstance().compare(this.kv, o.kv);
153    }
154
155    public static class CellWritableComparator extends WritableComparator {
156
157      @Override
158      public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
159        try {
160          CellWritableComparable kv1 = new CellWritableComparable();
161          kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1)));
162          CellWritableComparable kv2 = new CellWritableComparable();
163          kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2)));
164          return compare(kv1, kv2);
165        } catch (IOException e) {
166          throw new RuntimeException(e);
167        }
168      }
169
170    }
171
172  }
173
174  public static class CellReducer
175      extends
176      Reducer<CellWritableComparable, Cell, ImmutableBytesWritable, Cell> {
177    protected void reduce(
178        CellWritableComparable row,
179        Iterable<Cell> kvs,
180        Reducer<CellWritableComparable,
181          Cell, ImmutableBytesWritable, Cell>.Context context)
182        throws java.io.IOException, InterruptedException {
183      int index = 0;
184      for (Cell kv : kvs) {
185        context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)),
186          new MapReduceExtendedCell(kv));
187        if (++index % 100 == 0)
188          context.setStatus("Wrote " + index + " KeyValues, "
189              + "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
190      }
191    }
192  }
193
194  public static class CellSortImporter
195      extends TableMapper<CellWritableComparable, Cell> {
196    private Map<byte[], byte[]> cfRenameMap;
197    private Filter filter;
198    private static final Logger LOG = LoggerFactory.getLogger(CellImporter.class);
199
200    /**
201     * @param row  The current table row key.
202     * @param value  The columns.
203     * @param context  The current context.
204     * @throws IOException When something is broken with the data.
205     */
206    @Override
207    public void map(ImmutableBytesWritable row, Result value,
208      Context context)
209    throws IOException {
210      try {
211        if (LOG.isTraceEnabled()) {
212          LOG.trace("Considering the row."
213              + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
214        }
215        if (filter == null || !filter.filterRowKey(
216          PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))) {
217          for (Cell kv : value.rawCells()) {
218            kv = filterKv(filter, kv);
219            // skip if we filtered it out
220            if (kv == null) continue;
221            Cell ret = convertKv(kv, cfRenameMap);
222            context.write(new CellWritableComparable(ret), ret);
223          }
224        }
225      } catch (InterruptedException e) {
226        LOG.error("Interrupted while emitting Cell", e);
227        Thread.currentThread().interrupt();
228      }
229    }
230
231    @Override
232    public void setup(Context context) throws IOException {
233      cfRenameMap = createCfRenameMap(context.getConfiguration());
234      filter = instantiateFilter(context.getConfiguration());
235      int reduceNum = context.getNumReduceTasks();
236      Configuration conf = context.getConfiguration();
237      TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME));
238      try (Connection conn = ConnectionFactory.createConnection(conf);
239          RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
240        byte[][] startKeys = regionLocator.getStartKeys();
241        if (startKeys.length != reduceNum) {
242          throw new IOException("Region split after job initialization");
243        }
244        CellWritableComparable[] startKeyWraps =
245            new CellWritableComparable[startKeys.length - 1];
246        for (int i = 1; i < startKeys.length; ++i) {
247          startKeyWraps[i - 1] =
248              new CellWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i]));
249        }
250        CellWritableComparablePartitioner.START_KEYS = startKeyWraps;
251      }
252    }
253  }
254
255  /**
256   * A mapper that just writes out KeyValues.
257   */
258  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
259      justification="Writables are going away and this has been this way forever")
260  public static class CellImporter extends TableMapper<ImmutableBytesWritable, Cell> {
261    private Map<byte[], byte[]> cfRenameMap;
262    private Filter filter;
263    private static final Logger LOG = LoggerFactory.getLogger(CellImporter.class);
264
265    /**
266     * @param row  The current table row key.
267     * @param value  The columns.
268     * @param context  The current context.
269     * @throws IOException When something is broken with the data.
270     */
271    @Override
272    public void map(ImmutableBytesWritable row, Result value,
273      Context context)
274    throws IOException {
275      try {
276        if (LOG.isTraceEnabled()) {
277          LOG.trace("Considering the row."
278              + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
279        }
280        if (filter == null
281            || !filter.filterRowKey(PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(),
282                (short) row.getLength()))) {
283          for (Cell kv : value.rawCells()) {
284            kv = filterKv(filter, kv);
285            // skip if we filtered it out
286            if (kv == null) continue;
287            context.write(row, new MapReduceExtendedCell(convertKv(kv, cfRenameMap)));
288          }
289        }
290      } catch (InterruptedException e) {
291        LOG.error("Interrupted while emitting Cell", e);
292        Thread.currentThread().interrupt();
293      }
294    }
295
296    @Override
297    public void setup(Context context) {
298      cfRenameMap = createCfRenameMap(context.getConfiguration());
299      filter = instantiateFilter(context.getConfiguration());
300    }
301  }
302
303  /**
304   * Write table content out to files in hdfs.
305   */
306  public static class Importer extends TableMapper<ImmutableBytesWritable, Mutation> {
307    private Map<byte[], byte[]> cfRenameMap;
308    private List<UUID> clusterIds;
309    private Filter filter;
310    private Durability durability;
311
312    /**
313     * @param row  The current table row key.
314     * @param value  The columns.
315     * @param context  The current context.
316     * @throws IOException When something is broken with the data.
317     */
318    @Override
319    public void map(ImmutableBytesWritable row, Result value,
320      Context context)
321    throws IOException {
322      try {
323        writeResult(row, value, context);
324      } catch (InterruptedException e) {
325        LOG.error("Interrupted while writing result", e);
326        Thread.currentThread().interrupt();
327      }
328    }
329
330    private void writeResult(ImmutableBytesWritable key, Result result, Context context)
331    throws IOException, InterruptedException {
332      Put put = null;
333      Delete delete = null;
334      if (LOG.isTraceEnabled()) {
335        LOG.trace("Considering the row."
336            + Bytes.toString(key.get(), key.getOffset(), key.getLength()));
337      }
338      if (filter == null
339          || !filter.filterRowKey(PrivateCellUtil.createFirstOnRow(key.get(), key.getOffset(),
340              (short) key.getLength()))) {
341        processKV(key, result, context, put, delete);
342      }
343    }
344
345    protected void processKV(ImmutableBytesWritable key, Result result, Context context, Put put,
346        Delete delete) throws IOException, InterruptedException {
347      for (Cell kv : result.rawCells()) {
348        kv = filterKv(filter, kv);
349        // skip if we filter it out
350        if (kv == null) continue;
351
352        kv = convertKv(kv, cfRenameMap);
353        // Deletes and Puts are gathered and written when finished
354        /*
355         * If there are sequence of mutations and tombstones in an Export, and after Import the same
356         * sequence should be restored as it is. If we combine all Delete tombstones into single
357         * request then there is chance of ignoring few DeleteFamily tombstones, because if we
358         * submit multiple DeleteFamily tombstones in single Delete request then we are maintaining
359         * only newest in hbase table and ignoring other. Check - HBASE-12065
360         */
361        if (PrivateCellUtil.isDeleteFamily(kv)) {
362          Delete deleteFamily = new Delete(key.get());
363          deleteFamily.add(kv);
364          if (durability != null) {
365            deleteFamily.setDurability(durability);
366          }
367          deleteFamily.setClusterIds(clusterIds);
368          context.write(key, deleteFamily);
369        } else if (CellUtil.isDelete(kv)) {
370          if (delete == null) {
371            delete = new Delete(key.get());
372          }
373          delete.add(kv);
374        } else {
375          if (put == null) {
376            put = new Put(key.get());
377          }
378          addPutToKv(put, kv);
379        }
380      }
381      if (put != null) {
382        if (durability != null) {
383          put.setDurability(durability);
384        }
385        put.setClusterIds(clusterIds);
386        context.write(key, put);
387      }
388      if (delete != null) {
389        if (durability != null) {
390          delete.setDurability(durability);
391        }
392        delete.setClusterIds(clusterIds);
393        context.write(key, delete);
394      }
395    }
396
397    protected void addPutToKv(Put put, Cell kv) throws IOException {
398      put.add(kv);
399    }
400
401    @Override
402    public void setup(Context context) {
403      LOG.info("Setting up " + getClass() + " mapper.");
404      Configuration conf = context.getConfiguration();
405      cfRenameMap = createCfRenameMap(conf);
406      filter = instantiateFilter(conf);
407      String durabilityStr = conf.get(WAL_DURABILITY);
408      if(durabilityStr != null){
409        durability = Durability.valueOf(durabilityStr.toUpperCase(Locale.ROOT));
410        LOG.info("setting WAL durability to " + durability);
411      } else {
412        LOG.info("setting WAL durability to default.");
413      }
414      // TODO: This is kind of ugly doing setup of ZKW just to read the clusterid.
415      ZKWatcher zkw = null;
416      Exception ex = null;
417      try {
418        zkw = new ZKWatcher(conf, context.getTaskAttemptID().toString(), null);
419        clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
420      } catch (ZooKeeperConnectionException e) {
421        ex = e;
422        LOG.error("Problem connecting to ZooKeper during task setup", e);
423      } catch (KeeperException e) {
424        ex = e;
425        LOG.error("Problem reading ZooKeeper data during task setup", e);
426      } catch (IOException e) {
427        ex = e;
428        LOG.error("Problem setting up task", e);
429      } finally {
430        if (zkw != null) zkw.close();
431      }
432      if (clusterIds == null) {
433        // exit early if setup fails
434        throw new RuntimeException(ex);
435      }
436    }
437  }
438
439  /**
440   * Create a {@link Filter} to apply to all incoming keys ({@link KeyValue KeyValues}) to
441   * optionally not include in the job output
442   * @param conf {@link Configuration} from which to load the filter
443   * @return the filter to use for the task, or <tt>null</tt> if no filter to should be used
444   * @throws IllegalArgumentException if the filter is misconfigured
445   */
446  public static Filter instantiateFilter(Configuration conf) {
447    // get the filter, if it was configured
448    Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
449    if (filterClass == null) {
450      LOG.debug("No configured filter class, accepting all keyvalues.");
451      return null;
452    }
453    LOG.debug("Attempting to create filter:" + filterClass);
454    String[] filterArgs = conf.getStrings(FILTER_ARGS_CONF_KEY);
455    ArrayList<byte[]> quotedArgs = toQuotedByteArrays(filterArgs);
456    try {
457      Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
458      return (Filter) m.invoke(null, quotedArgs);
459    } catch (IllegalAccessException e) {
460      LOG.error("Couldn't instantiate filter!", e);
461      throw new RuntimeException(e);
462    } catch (SecurityException e) {
463      LOG.error("Couldn't instantiate filter!", e);
464      throw new RuntimeException(e);
465    } catch (NoSuchMethodException e) {
466      LOG.error("Couldn't instantiate filter!", e);
467      throw new RuntimeException(e);
468    } catch (IllegalArgumentException e) {
469      LOG.error("Couldn't instantiate filter!", e);
470      throw new RuntimeException(e);
471    } catch (InvocationTargetException e) {
472      LOG.error("Couldn't instantiate filter!", e);
473      throw new RuntimeException(e);
474    }
475  }
476
477  private static ArrayList<byte[]> toQuotedByteArrays(String... stringArgs) {
478    ArrayList<byte[]> quotedArgs = new ArrayList<>();
479    for (String stringArg : stringArgs) {
480      // all the filters' instantiation methods expected quoted args since they are coming from
481      // the shell, so add them here, though it shouldn't really be needed :-/
482      quotedArgs.add(Bytes.toBytes("'" + stringArg + "'"));
483    }
484    return quotedArgs;
485  }
486
487  /**
488   * Attempt to filter out the keyvalue
489   * @param c {@link Cell} on which to apply the filter
490   * @return <tt>null</tt> if the key should not be written, otherwise returns the original
491   *         {@link Cell}
492   */
493  public static Cell filterKv(Filter filter, Cell c) throws IOException {
494    // apply the filter and skip this kv if the filter doesn't apply
495    if (filter != null) {
496      Filter.ReturnCode code = filter.filterCell(c);
497      if (LOG.isTraceEnabled()) {
498        LOG.trace("Filter returned:" + code + " for the cell:" + c);
499      }
500      // if its not an accept type, then skip this kv
501      if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
502          .equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
503        return null;
504      }
505    }
506    return c;
507  }
508
509  // helper: create a new KeyValue based on CF rename map
510  private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {
511    if(cfRenameMap != null) {
512      // If there's a rename mapping for this CF, create a new KeyValue
513      byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
514      if (newCfName != null) {
515        List<Tag> tags = PrivateCellUtil.getTags(kv);
516        kv = new KeyValue(kv.getRowArray(), // row buffer
517            kv.getRowOffset(),              // row offset
518            kv.getRowLength(),              // row length
519            newCfName,                      // CF buffer
520            0,                              // CF offset
521            newCfName.length,               // CF length
522            kv.getQualifierArray(),         // qualifier buffer
523            kv.getQualifierOffset(),        // qualifier offset
524            kv.getQualifierLength(),        // qualifier length
525            kv.getTimestamp(),              // timestamp
526            KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type
527            kv.getValueArray(),             // value buffer
528            kv.getValueOffset(),            // value offset
529            kv.getValueLength(),           // value length
530            tags.size() == 0 ? null: tags);
531      }
532    }
533    return kv;
534  }
535
536  // helper: make a map from sourceCfName to destCfName by parsing a config key
537  private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) {
538    Map<byte[], byte[]> cfRenameMap = null;
539    String allMappingsPropVal = conf.get(CF_RENAME_PROP);
540    if(allMappingsPropVal != null) {
541      // The conf value format should be sourceCf1:destCf1,sourceCf2:destCf2,...
542      String[] allMappings = allMappingsPropVal.split(",");
543      for (String mapping: allMappings) {
544        if(cfRenameMap == null) {
545            cfRenameMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
546        }
547        String [] srcAndDest = mapping.split(":");
548        if(srcAndDest.length != 2) {
549            continue;
550        }
551        cfRenameMap.put(Bytes.toBytes(srcAndDest[0]), Bytes.toBytes(srcAndDest[1]));
552      }
553    }
554    return cfRenameMap;
555  }
556
557  /**
558   * <p>Sets a configuration property with key {@link #CF_RENAME_PROP} in conf that tells
559   * the mapper how to rename column families.
560   *
561   * <p>Alternately, instead of calling this function, you could set the configuration key
562   * {@link #CF_RENAME_PROP} yourself. The value should look like
563   * <pre>srcCf1:destCf1,srcCf2:destCf2,....</pre>. This would have the same effect on
564   * the mapper behavior.
565   *
566   * @param conf the Configuration in which the {@link #CF_RENAME_PROP} key will be
567   *  set
568   * @param renameMap a mapping from source CF names to destination CF names
569   */
570  static public void configureCfRenaming(Configuration conf,
571          Map<String, String> renameMap) {
572    StringBuilder sb = new StringBuilder();
573    for(Map.Entry<String,String> entry: renameMap.entrySet()) {
574      String sourceCf = entry.getKey();
575      String destCf = entry.getValue();
576
577      if(sourceCf.contains(":") || sourceCf.contains(",") ||
578              destCf.contains(":") || destCf.contains(",")) {
579        throw new IllegalArgumentException("Illegal character in CF names: "
580              + sourceCf + ", " + destCf);
581      }
582
583      if(sb.length() != 0) {
584        sb.append(",");
585      }
586      sb.append(sourceCf + ":" + destCf);
587    }
588    conf.set(CF_RENAME_PROP, sb.toString());
589  }
590
591  /**
592   * Add a Filter to be instantiated on import
593   * @param conf Configuration to update (will be passed to the job)
594   * @param clazz {@link Filter} subclass to instantiate on the server.
595   * @param filterArgs List of arguments to pass to the filter on instantiation
596   */
597  public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz,
598      List<String> filterArgs) throws IOException {
599    conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName());
600    conf.setStrings(Import.FILTER_ARGS_CONF_KEY, filterArgs.toArray(new String[filterArgs.size()]));
601  }
602
603  /**
604   * Sets up the actual job.
605   * @param conf The current configuration.
606   * @param args The command line parameters.
607   * @return The newly created job.
608   * @throws IOException When setting up the job fails.
609   */
610  public static Job createSubmittableJob(Configuration conf, String[] args)
611  throws IOException {
612    TableName tableName = TableName.valueOf(args[0]);
613    conf.set(TABLE_NAME, tableName.getNameAsString());
614    Path inputDir = new Path(args[1]);
615    Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
616    job.setJarByClass(Importer.class);
617    FileInputFormat.setInputPaths(job, inputDir);
618    job.setInputFormatClass(SequenceFileInputFormat.class);
619    String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
620
621    // make sure we get the filter in the jars
622    try {
623      Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
624      if (filter != null) {
625        TableMapReduceUtil.addDependencyJarsForClasses(conf, filter);
626      }
627    } catch (Exception e) {
628      throw new IOException(e);
629    }
630
631    if (hfileOutPath != null && conf.getBoolean(HAS_LARGE_RESULT, false)) {
632      LOG.info("Use Large Result!!");
633      try (Connection conn = ConnectionFactory.createConnection(conf);
634          Table table = conn.getTable(tableName);
635          RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
636        HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
637        job.setMapperClass(CellSortImporter.class);
638        job.setReducerClass(CellReducer.class);
639        Path outputDir = new Path(hfileOutPath);
640        FileOutputFormat.setOutputPath(job, outputDir);
641        job.setMapOutputKeyClass(CellWritableComparable.class);
642        job.setMapOutputValueClass(MapReduceExtendedCell.class);
643        job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
644            CellWritableComparable.CellWritableComparator.class,
645            RawComparator.class);
646        Path partitionsPath =
647            new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration()));
648        FileSystem fs = FileSystem.get(job.getConfiguration());
649        fs.deleteOnExit(partitionsPath);
650        job.setPartitionerClass(CellWritableComparablePartitioner.class);
651        job.setNumReduceTasks(regionLocator.getStartKeys().length);
652        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
653            org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class);
654      }
655    } else if (hfileOutPath != null) {
656      LOG.info("writing to hfiles for bulk load.");
657      job.setMapperClass(CellImporter.class);
658      try (Connection conn = ConnectionFactory.createConnection(conf);
659          Table table = conn.getTable(tableName);
660          RegionLocator regionLocator = conn.getRegionLocator(tableName)){
661        job.setReducerClass(CellSortReducer.class);
662        Path outputDir = new Path(hfileOutPath);
663        FileOutputFormat.setOutputPath(job, outputDir);
664        job.setMapOutputKeyClass(ImmutableBytesWritable.class);
665        job.setMapOutputValueClass(MapReduceExtendedCell.class);
666        HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
667        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
668            org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class);
669      }
670    } else {
671      LOG.info("writing directly to table from Mapper.");
672      // No reducers.  Just write straight to table.  Call initTableReducerJob
673      // because it sets up the TableOutputFormat.
674      job.setMapperClass(Importer.class);
675      TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job);
676      job.setNumReduceTasks(0);
677    }
678    return job;
679  }
680
681  /*
682   * @param errorMsg Error message.  Can be null.
683   */
684  private static void usage(final String errorMsg) {
685    if (errorMsg != null && errorMsg.length() > 0) {
686      System.err.println("ERROR: " + errorMsg);
687    }
688    System.err.println("Usage: Import [options] <tablename> <inputdir>");
689    System.err.println("By default Import will load data directly into HBase. To instead generate");
690    System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
691    System.err.println("  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
692    System.err.println("If there is a large result that includes too much Cell "
693        + "whitch can occur OOME caused by the memery sort in reducer, pass the option:");
694    System.err.println("  -D" + HAS_LARGE_RESULT + "=true");
695    System.err
696        .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
697    System.err.println("  -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");
698    System.err.println("  -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");
699    System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "
700        + CF_RENAME_PROP + " property. Futher, filters will only use the"
701        + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "
702        + " whether the current row needs to be ignored completely for processing and "
703        + " Filter#filterCell(Cell) method to determine if the Cell should be added;"
704        + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including"
705        + " the Cell.");
706    System.err.println("To import data exported from HBase 0.94, use");
707    System.err.println("  -Dhbase.import.version=0.94");
708    System.err.println("  -D " + JOB_NAME_CONF_KEY
709        + "=jobName - use the specified mapreduce job name for the import");
710    System.err.println("For performance consider the following options:\n"
711        + "  -Dmapreduce.map.speculative=false\n"
712        + "  -Dmapreduce.reduce.speculative=false\n"
713        + "  -D" + WAL_DURABILITY + "=<Used while writing data to hbase."
714            +" Allowed values are the supported durability values"
715            +" like SKIP_WAL/ASYNC_WAL/SYNC_WAL/...>");
716  }
717
718  /**
719   * If the durability is set to {@link Durability#SKIP_WAL} and the data is imported to hbase, we
720   * need to flush all the regions of the table as the data is held in memory and is also not
721   * present in the Write Ahead Log to replay in scenarios of a crash. This method flushes all the
722   * regions of the table in the scenarios of import data to hbase with {@link Durability#SKIP_WAL}
723   */
724  public static void flushRegionsIfNecessary(Configuration conf) throws IOException,
725      InterruptedException {
726    String tableName = conf.get(TABLE_NAME);
727    Admin hAdmin = null;
728    Connection connection = null;
729    String durability = conf.get(WAL_DURABILITY);
730    // Need to flush if the data is written to hbase and skip wal is enabled.
731    if (conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null
732        && Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) {
733      LOG.info("Flushing all data that skipped the WAL.");
734      try {
735        connection = ConnectionFactory.createConnection(conf);
736        hAdmin = connection.getAdmin();
737        hAdmin.flush(TableName.valueOf(tableName));
738      } finally {
739        if (hAdmin != null) {
740          hAdmin.close();
741        }
742        if (connection != null) {
743          connection.close();
744        }
745      }
746    }
747  }
748
749  @Override
750  public int run(String[] args) throws Exception {
751    if (args.length < 2) {
752      usage("Wrong number of arguments: " + args.length);
753      return -1;
754    }
755    String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER);
756    if (inputVersionString != null) {
757      getConf().set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString);
758    }
759    Job job = createSubmittableJob(getConf(), args);
760    boolean isJobSuccessful = job.waitForCompletion(true);
761    if(isJobSuccessful){
762      // Flush all the regions of the table
763      flushRegionsIfNecessary(getConf());
764    }
765    long inputRecords = job.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue();
766    long outputRecords = job.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS).getValue();
767    if (outputRecords < inputRecords) {
768      System.err.println("Warning, not all records were imported (maybe filtered out).");
769      if (outputRecords == 0) {
770        System.err.println("If the data was exported from HBase 0.94 "+
771            "consider using -Dhbase.import.version=0.94.");
772      }
773    }
774
775    return (isJobSuccessful ? 0 : 1);
776  }
777
778  /**
779   * Main entry point.
780   * @param args The command line parameters.
781   * @throws Exception When running the job fails.
782   */
783  public static void main(String[] args) throws Exception {
784    int errCode = ToolRunner.run(HBaseConfiguration.create(), new Import(), args);
785    System.exit(errCode);
786  }
787
788}