View Javadoc

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