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.IOException;
22  import java.lang.reflect.InvocationTargetException;
23  import java.lang.reflect.Method;
24  import java.util.ArrayList;
25  import java.util.Collections;
26  import java.util.List;
27  import java.util.Map;
28  import java.util.TreeMap;
29  import java.util.UUID;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.hbase.classification.InterfaceAudience;
34  import org.apache.hadoop.hbase.classification.InterfaceStability;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.hbase.Cell;
38  import org.apache.hadoop.hbase.CellUtil;
39  import org.apache.hadoop.hbase.HBaseConfiguration;
40  import org.apache.hadoop.hbase.KeyValue;
41  import org.apache.hadoop.hbase.KeyValueUtil;
42  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
43  import org.apache.hadoop.hbase.client.Delete;
44  import org.apache.hadoop.hbase.client.Durability;
45  import org.apache.hadoop.hbase.client.HBaseAdmin;
46  import org.apache.hadoop.hbase.client.HTable;
47  import org.apache.hadoop.hbase.client.Mutation;
48  import org.apache.hadoop.hbase.client.Put;
49  import org.apache.hadoop.hbase.client.Result;
50  import org.apache.hadoop.hbase.filter.Filter;
51  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
52  import org.apache.hadoop.hbase.util.Bytes;
53  import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
54  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
55  import org.apache.hadoop.mapreduce.Job;
56  import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
57  import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
58  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
59  import org.apache.hadoop.util.GenericOptionsParser;
60  import org.apache.zookeeper.KeeperException;
61  
62  
63  /**
64   * Import data written by {@link Export}.
65   */
66  @InterfaceAudience.Public
67  @InterfaceStability.Stable
68  public class Import {
69    private static final Log LOG = LogFactory.getLog(Import.class);
70    final static String NAME = "import";
71    public final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
72    public final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
73    public final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
74    public final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
75    public final static String TABLE_NAME = "import.table.name";
76    public final static String WAL_DURABILITY = "import.wal.durability";
77  
78    /**
79     * A mapper that just writes out KeyValues.
80     */
81    public static class KeyValueImporter extends TableMapper<ImmutableBytesWritable, KeyValue> {
82      private Map<byte[], byte[]> cfRenameMap;
83      private Filter filter;
84      private static final Log LOG = LogFactory.getLog(KeyValueImporter.class);
85  
86      /**
87       * @param row  The current table row key.
88       * @param value  The columns.
89       * @param context  The current context.
90       * @throws IOException When something is broken with the data.
91       */
92      @Override
93      public void map(ImmutableBytesWritable row, Result value,
94        Context context)
95      throws IOException {
96        try {
97          if (LOG.isTraceEnabled()) {
98            LOG.trace("Considering the row."
99                + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
100         }
101         if (filter == null || !filter.filterRowKey(row.get(), row.getOffset(), row.getLength())) {
102           for (Cell kv : value.rawCells()) {
103             kv = filterKv(filter, kv);
104             // skip if we filtered it out
105             if (kv == null) continue;
106             // TODO get rid of ensureKeyValue
107             context.write(row, KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap)));
108           }
109         }
110       } catch (InterruptedException e) {
111         e.printStackTrace();
112       }
113     }
114 
115     @Override
116     public void setup(Context context) {
117       cfRenameMap = createCfRenameMap(context.getConfiguration());
118       filter = instantiateFilter(context.getConfiguration());
119     }
120   }
121 
122   /**
123    * Write table content out to files in hdfs.
124    */
125   public static class Importer extends TableMapper<ImmutableBytesWritable, Mutation> {
126     private Map<byte[], byte[]> cfRenameMap;
127     private List<UUID> clusterIds;
128     private Filter filter;
129     private Durability durability;
130 
131     /**
132      * @param row  The current table row key.
133      * @param value  The columns.
134      * @param context  The current context.
135      * @throws IOException When something is broken with the data.
136      */
137     @Override
138     public void map(ImmutableBytesWritable row, Result value,
139       Context context)
140     throws IOException {
141       try {
142         writeResult(row, value, context);
143       } catch (InterruptedException e) {
144         e.printStackTrace();
145       }
146     }
147 
148     private void writeResult(ImmutableBytesWritable key, Result result, Context context)
149     throws IOException, InterruptedException {
150       Put put = null;
151       Delete delete = null;
152       if (LOG.isTraceEnabled()) {
153         LOG.trace("Considering the row."
154             + Bytes.toString(key.get(), key.getOffset(), key.getLength()));
155       }
156       if (filter == null || !filter.filterRowKey(key.get(), key.getOffset(), key.getLength())) {
157         processKV(key, result, context, put, delete);
158       }
159     }
160 
161     protected void processKV(ImmutableBytesWritable key, Result result, Context context, Put put,
162         Delete delete) throws IOException, InterruptedException {
163       for (Cell kv : result.rawCells()) {
164         kv = filterKv(filter, kv);
165         // skip if we filter it out
166         if (kv == null) continue;
167 
168         kv = convertKv(kv, cfRenameMap);
169         // Deletes and Puts are gathered and written when finished
170         /*
171          * If there are sequence of mutations and tombstones in an Export, and after Import the same
172          * sequence should be restored as it is. If we combine all Delete tombstones into single
173          * request then there is chance of ignoring few DeleteFamily tombstones, because if we
174          * submit multiple DeleteFamily tombstones in single Delete request then we are maintaining
175          * only newest in hbase table and ignoring other. Check - HBASE-12065
176          */
177         if (CellUtil.isDeleteFamily(kv)) {
178           Delete deleteFamily = new Delete(key.get());
179           deleteFamily.addDeleteMarker(kv);
180           if (durability != null) {
181             deleteFamily.setDurability(durability);
182           }
183           deleteFamily.setClusterIds(clusterIds);
184           context.write(key, deleteFamily);
185         } else if (CellUtil.isDelete(kv)) {
186           if (delete == null) {
187             delete = new Delete(key.get());
188           }
189           delete.addDeleteMarker(kv);
190         } else {
191           if (put == null) {
192             put = new Put(key.get());
193           }
194           addPutToKv(put, kv);
195         }
196       }
197       if (put != null) {
198         if (durability != null) {
199           put.setDurability(durability);
200         }
201         put.setClusterIds(clusterIds);
202         context.write(key, put);
203       }
204       if (delete != null) {
205         if (durability != null) {
206           delete.setDurability(durability);
207         }
208         delete.setClusterIds(clusterIds);
209         context.write(key, delete);
210       }
211     }
212 
213     protected void addPutToKv(Put put, Cell kv) throws IOException {
214       put.add(kv);
215     }
216 
217     @Override
218     public void setup(Context context) {
219       Configuration conf = context.getConfiguration();
220       cfRenameMap = createCfRenameMap(conf);
221       filter = instantiateFilter(conf);
222       String durabilityStr = conf.get(WAL_DURABILITY);
223       if(durabilityStr != null){
224         durability = Durability.valueOf(durabilityStr.toUpperCase());
225       }
226       // TODO: This is kind of ugly doing setup of ZKW just to read the clusterid.
227       ZooKeeperWatcher zkw = null;
228       try {
229         zkw = new ZooKeeperWatcher(conf, context.getTaskAttemptID().toString(), null);
230         clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
231       } catch (ZooKeeperConnectionException e) {
232         LOG.error("Problem connecting to ZooKeper during task setup", e);
233       } catch (KeeperException e) {
234         LOG.error("Problem reading ZooKeeper data during task setup", e);
235       } catch (IOException e) {
236         LOG.error("Problem setting up task", e);
237       } finally {
238         if (zkw != null) zkw.close();
239       }
240     }
241   }
242 
243   /**
244    * Create a {@link Filter} to apply to all incoming keys ({@link KeyValue KeyValues}) to
245    * optionally not include in the job output
246    * @param conf {@link Configuration} from which to load the filter
247    * @return the filter to use for the task, or <tt>null</tt> if no filter to should be used
248    * @throws IllegalArgumentException if the filter is misconfigured
249    */
250   public static Filter instantiateFilter(Configuration conf) {
251     // get the filter, if it was configured    
252     Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
253     if (filterClass == null) {
254       LOG.debug("No configured filter class, accepting all keyvalues.");
255       return null;
256     }
257     LOG.debug("Attempting to create filter:" + filterClass);
258     String[] filterArgs = conf.getStrings(FILTER_ARGS_CONF_KEY);
259     ArrayList<byte[]> quotedArgs = toQuotedByteArrays(filterArgs);
260     try {
261       Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
262       return (Filter) m.invoke(null, quotedArgs);
263     } catch (IllegalAccessException e) {
264       LOG.error("Couldn't instantiate filter!", e);
265       throw new RuntimeException(e);
266     } catch (SecurityException e) {
267       LOG.error("Couldn't instantiate filter!", e);
268       throw new RuntimeException(e);
269     } catch (NoSuchMethodException e) {
270       LOG.error("Couldn't instantiate filter!", e);
271       throw new RuntimeException(e);
272     } catch (IllegalArgumentException e) {
273       LOG.error("Couldn't instantiate filter!", e);
274       throw new RuntimeException(e);
275     } catch (InvocationTargetException e) {
276       LOG.error("Couldn't instantiate filter!", e);
277       throw new RuntimeException(e);
278     }
279   }
280 
281   private static ArrayList<byte[]> toQuotedByteArrays(String... stringArgs) {
282     ArrayList<byte[]> quotedArgs = new ArrayList<byte[]>();
283     for (String stringArg : stringArgs) {
284       // all the filters' instantiation methods expected quoted args since they are coming from
285       // the shell, so add them here, though it shouldn't really be needed :-/
286       quotedArgs.add(Bytes.toBytes("'" + stringArg + "'"));
287     }
288     return quotedArgs;
289   }
290 
291   /**
292    * Attempt to filter out the keyvalue
293    * @param kv {@link KeyValue} on which to apply the filter
294    * @return <tt>null</tt> if the key should not be written, otherwise returns the original
295    *         {@link KeyValue}
296    */
297   public static Cell filterKv(Filter filter, Cell kv) throws IOException {
298     // apply the filter and skip this kv if the filter doesn't apply
299     if (filter != null) {
300       Filter.ReturnCode code = filter.filterKeyValue(kv);
301       if (LOG.isTraceEnabled()) {
302         LOG.trace("Filter returned:" + code + " for the key value:" + kv);
303       }
304       // if its not an accept type, then skip this kv
305       if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
306           .equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
307         return null;
308       }
309     }
310     return kv;
311   }
312 
313   // helper: create a new KeyValue based on CF rename map
314   private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {
315     if(cfRenameMap != null) {
316       // If there's a rename mapping for this CF, create a new KeyValue
317       byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
318       if(newCfName != null) {
319           kv = new KeyValue(kv.getRowArray(), // row buffer 
320                   kv.getRowOffset(),        // row offset
321                   kv.getRowLength(),        // row length
322                   newCfName,                // CF buffer
323                   0,                        // CF offset 
324                   newCfName.length,         // CF length 
325                   kv.getQualifierArray(),   // qualifier buffer
326                   kv.getQualifierOffset(),  // qualifier offset
327                   kv.getQualifierLength(),  // qualifier length
328                   kv.getTimestamp(),        // timestamp
329                   KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type
330                   kv.getValueArray(),       // value buffer 
331                   kv.getValueOffset(),      // value offset
332                   kv.getValueLength());     // value length
333       }
334     }
335     return kv;
336   }
337 
338   // helper: make a map from sourceCfName to destCfName by parsing a config key
339   private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) {
340     Map<byte[], byte[]> cfRenameMap = null;
341     String allMappingsPropVal = conf.get(CF_RENAME_PROP);
342     if(allMappingsPropVal != null) {
343       // The conf value format should be sourceCf1:destCf1,sourceCf2:destCf2,...
344       String[] allMappings = allMappingsPropVal.split(",");
345       for (String mapping: allMappings) {
346         if(cfRenameMap == null) {
347             cfRenameMap = new TreeMap<byte[],byte[]>(Bytes.BYTES_COMPARATOR);
348         }
349         String [] srcAndDest = mapping.split(":");
350         if(srcAndDest.length != 2) {
351             continue;
352         }
353         cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
354       }
355     }
356     return cfRenameMap;
357   }
358 
359   /**
360    * <p>Sets a configuration property with key {@link #CF_RENAME_PROP} in conf that tells
361    * the mapper how to rename column families.
362    * 
363    * <p>Alternately, instead of calling this function, you could set the configuration key 
364    * {@link #CF_RENAME_PROP} yourself. The value should look like 
365    * <pre>srcCf1:destCf1,srcCf2:destCf2,....</pre>. This would have the same effect on
366    * the mapper behavior.
367    * 
368    * @param conf the Configuration in which the {@link #CF_RENAME_PROP} key will be
369    *  set
370    * @param renameMap a mapping from source CF names to destination CF names
371    */
372   static public void configureCfRenaming(Configuration conf, 
373           Map<String, String> renameMap) {
374     StringBuilder sb = new StringBuilder();
375     for(Map.Entry<String,String> entry: renameMap.entrySet()) {
376       String sourceCf = entry.getKey();
377       String destCf = entry.getValue();
378 
379       if(sourceCf.contains(":") || sourceCf.contains(",") || 
380               destCf.contains(":") || destCf.contains(",")) {
381         throw new IllegalArgumentException("Illegal character in CF names: " 
382               + sourceCf + ", " + destCf);
383       }
384 
385       if(sb.length() != 0) {
386         sb.append(",");
387       }
388       sb.append(sourceCf + ":" + destCf);
389     }
390     conf.set(CF_RENAME_PROP, sb.toString());
391   }
392 
393   /**
394    * Add a Filter to be instantiated on import
395    * @param conf Configuration to update (will be passed to the job)
396    * @param clazz {@link Filter} subclass to instantiate on the server.
397    * @param filterArgs List of arguments to pass to the filter on instantiation
398    */
399   public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz,
400       List<String> filterArgs) throws IOException {
401     conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName());
402     conf.setStrings(Import.FILTER_ARGS_CONF_KEY, filterArgs.toArray(new String[filterArgs.size()]));
403   }
404 
405   /**
406    * Sets up the actual job.
407    * @param conf The current configuration.
408    * @param args The command line parameters.
409    * @return The newly created job.
410    * @throws IOException When setting up the job fails.
411    */
412   public static Job createSubmittableJob(Configuration conf, String[] args)
413   throws IOException {
414     String tableName = args[0];
415     conf.set(TABLE_NAME, tableName);
416     Path inputDir = new Path(args[1]);
417     Job job = new Job(conf, NAME + "_" + tableName);
418     job.setJarByClass(Importer.class);
419     FileInputFormat.setInputPaths(job, inputDir);
420     job.setInputFormatClass(SequenceFileInputFormat.class);
421     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
422 
423     // make sure we get the filter in the jars
424     try {
425       Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
426       if (filter != null) {
427         TableMapReduceUtil.addDependencyJars(conf, filter);
428       }
429     } catch (Exception e) {
430       throw new IOException(e);
431     }
432 
433     if (hfileOutPath != null) {
434       job.setMapperClass(KeyValueImporter.class);
435       HTable table = new HTable(conf, tableName);
436       job.setReducerClass(KeyValueSortReducer.class);
437       Path outputDir = new Path(hfileOutPath);
438       FileOutputFormat.setOutputPath(job, outputDir);
439       job.setMapOutputKeyClass(ImmutableBytesWritable.class);
440       job.setMapOutputValueClass(KeyValue.class);
441       HFileOutputFormat.configureIncrementalLoad(job, table);
442       TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
443           com.google.common.base.Preconditions.class);
444     } else {
445       // No reducers.  Just write straight to table.  Call initTableReducerJob
446       // because it sets up the TableOutputFormat.
447       job.setMapperClass(Importer.class);
448       TableMapReduceUtil.initTableReducerJob(tableName, null, job);
449       job.setNumReduceTasks(0);
450     }
451     return job;
452   }
453 
454   /*
455    * @param errorMsg Error message.  Can be null.
456    */
457   private static void usage(final String errorMsg) {
458     if (errorMsg != null && errorMsg.length() > 0) {
459       System.err.println("ERROR: " + errorMsg);
460     }
461     System.err.println("Usage: Import [options] <tablename> <inputdir>");
462     System.err.println("By default Import will load data directly into HBase. To instead generate");
463     System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
464     System.err.println("  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
465     System.err
466         .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
467     System.err.println("  -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");
468     System.err.println("  -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");
469     System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "
470         + CF_RENAME_PROP + " property. Futher, filters will only use the"
471         + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "
472         + " whether the current row needs to be ignored completely for processing and "
473         + " Filter#filterKeyValue(KeyValue) method to determine if the KeyValue should be added;"
474         + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including"
475         + " the KeyValue.");
476     System.err.println("For performance consider the following options:\n"
477         + "  -Dmapred.map.tasks.speculative.execution=false\n"
478         + "  -Dmapred.reduce.tasks.speculative.execution=false\n"
479         + "  -D" + WAL_DURABILITY + "=<Used while writing data to hbase."
480             +" Allowed values are the supported durability values"
481             +" like SKIP_WAL/ASYNC_WAL/SYNC_WAL/...>");
482   }
483 
484   /**
485    * If the durability is set to {@link Durability#SKIP_WAL} and the data is imported to hbase, we
486    * need to flush all the regions of the table as the data is held in memory and is also not
487    * present in the Write Ahead Log to replay in scenarios of a crash. This method flushes all the
488    * regions of the table in the scenarios of import data to hbase with {@link Durability#SKIP_WAL}
489    */
490   public static void flushRegionsIfNecessary(Configuration conf) throws IOException,
491       InterruptedException {
492     String tableName = conf.get(TABLE_NAME);
493     HBaseAdmin hAdmin = null;
494     String durability = conf.get(WAL_DURABILITY);
495     // Need to flush if the data is written to hbase and skip wal is enabled.
496     if (conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null
497         && Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) {
498       try {
499         hAdmin = new HBaseAdmin(conf);
500         hAdmin.flush(tableName);
501       } finally {
502         if (hAdmin != null) {
503           hAdmin.close();
504         }
505       }
506     }
507   }
508 
509   /**
510    * Main entry point.
511    *
512    * @param args  The command line parameters.
513    * @throws Exception When running the job fails.
514    */
515   public static void main(String[] args) throws Exception {
516     Configuration conf = HBaseConfiguration.create();
517     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
518     if (otherArgs.length < 2) {
519       usage("Wrong number of arguments: " + otherArgs.length);
520       System.exit(-1);
521     }
522     String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER);
523     if (inputVersionString != null) {
524       conf.set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString);
525     }
526     Job job = createSubmittableJob(conf, otherArgs);
527     boolean isJobSuccessful = job.waitForCompletion(true);
528     if(isJobSuccessful){
529       // Flush all the regions of the table
530       flushRegionsIfNecessary(conf);
531     }
532     System.exit(job.waitForCompletion(true) ? 0 : 1);
533   }
534 }