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       Exception ex = null;
229       try {
230         zkw = new ZooKeeperWatcher(conf, context.getTaskAttemptID().toString(), null);
231         clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
232       } catch (ZooKeeperConnectionException e) {
233         ex = e;
234         LOG.error("Problem connecting to ZooKeper during task setup", e);
235       } catch (KeeperException e) {
236         ex = e;
237         LOG.error("Problem reading ZooKeeper data during task setup", e);
238       } catch (IOException e) {
239         ex = e;
240         LOG.error("Problem setting up task", e);
241       } finally {
242         if (zkw != null) zkw.close();
243       }
244       if (clusterIds == null) {
245         // exit early if setup fails
246         throw new RuntimeException(ex);
247       }
248     }
249   }
250 
251   /**
252    * Create a {@link Filter} to apply to all incoming keys ({@link KeyValue KeyValues}) to
253    * optionally not include in the job output
254    * @param conf {@link Configuration} from which to load the filter
255    * @return the filter to use for the task, or <tt>null</tt> if no filter to should be used
256    * @throws IllegalArgumentException if the filter is misconfigured
257    */
258   public static Filter instantiateFilter(Configuration conf) {
259     // get the filter, if it was configured    
260     Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
261     if (filterClass == null) {
262       LOG.debug("No configured filter class, accepting all keyvalues.");
263       return null;
264     }
265     LOG.debug("Attempting to create filter:" + filterClass);
266     String[] filterArgs = conf.getStrings(FILTER_ARGS_CONF_KEY);
267     ArrayList<byte[]> quotedArgs = toQuotedByteArrays(filterArgs);
268     try {
269       Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
270       return (Filter) m.invoke(null, quotedArgs);
271     } catch (IllegalAccessException e) {
272       LOG.error("Couldn't instantiate filter!", e);
273       throw new RuntimeException(e);
274     } catch (SecurityException e) {
275       LOG.error("Couldn't instantiate filter!", e);
276       throw new RuntimeException(e);
277     } catch (NoSuchMethodException e) {
278       LOG.error("Couldn't instantiate filter!", e);
279       throw new RuntimeException(e);
280     } catch (IllegalArgumentException e) {
281       LOG.error("Couldn't instantiate filter!", e);
282       throw new RuntimeException(e);
283     } catch (InvocationTargetException e) {
284       LOG.error("Couldn't instantiate filter!", e);
285       throw new RuntimeException(e);
286     }
287   }
288 
289   private static ArrayList<byte[]> toQuotedByteArrays(String... stringArgs) {
290     ArrayList<byte[]> quotedArgs = new ArrayList<byte[]>();
291     for (String stringArg : stringArgs) {
292       // all the filters' instantiation methods expected quoted args since they are coming from
293       // the shell, so add them here, though it shouldn't really be needed :-/
294       quotedArgs.add(Bytes.toBytes("'" + stringArg + "'"));
295     }
296     return quotedArgs;
297   }
298 
299   /**
300    * Attempt to filter out the keyvalue
301    * @param kv {@link KeyValue} on which to apply the filter
302    * @return <tt>null</tt> if the key should not be written, otherwise returns the original
303    *         {@link KeyValue}
304    */
305   public static Cell filterKv(Filter filter, Cell kv) throws IOException {
306     // apply the filter and skip this kv if the filter doesn't apply
307     if (filter != null) {
308       Filter.ReturnCode code = filter.filterKeyValue(kv);
309       if (LOG.isTraceEnabled()) {
310         LOG.trace("Filter returned:" + code + " for the key value:" + kv);
311       }
312       // if its not an accept type, then skip this kv
313       if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
314           .equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
315         return null;
316       }
317     }
318     return kv;
319   }
320 
321   // helper: create a new KeyValue based on CF rename map
322   private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {
323     if(cfRenameMap != null) {
324       // If there's a rename mapping for this CF, create a new KeyValue
325       byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
326       if(newCfName != null) {
327           kv = new KeyValue(kv.getRowArray(), // row buffer 
328                   kv.getRowOffset(),        // row offset
329                   kv.getRowLength(),        // row length
330                   newCfName,                // CF buffer
331                   0,                        // CF offset 
332                   newCfName.length,         // CF length 
333                   kv.getQualifierArray(),   // qualifier buffer
334                   kv.getQualifierOffset(),  // qualifier offset
335                   kv.getQualifierLength(),  // qualifier length
336                   kv.getTimestamp(),        // timestamp
337                   KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type
338                   kv.getValueArray(),       // value buffer 
339                   kv.getValueOffset(),      // value offset
340                   kv.getValueLength());     // value length
341       }
342     }
343     return kv;
344   }
345 
346   // helper: make a map from sourceCfName to destCfName by parsing a config key
347   private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) {
348     Map<byte[], byte[]> cfRenameMap = null;
349     String allMappingsPropVal = conf.get(CF_RENAME_PROP);
350     if(allMappingsPropVal != null) {
351       // The conf value format should be sourceCf1:destCf1,sourceCf2:destCf2,...
352       String[] allMappings = allMappingsPropVal.split(",");
353       for (String mapping: allMappings) {
354         if(cfRenameMap == null) {
355             cfRenameMap = new TreeMap<byte[],byte[]>(Bytes.BYTES_COMPARATOR);
356         }
357         String [] srcAndDest = mapping.split(":");
358         if(srcAndDest.length != 2) {
359             continue;
360         }
361         cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
362       }
363     }
364     return cfRenameMap;
365   }
366 
367   /**
368    * <p>Sets a configuration property with key {@link #CF_RENAME_PROP} in conf that tells
369    * the mapper how to rename column families.
370    * 
371    * <p>Alternately, instead of calling this function, you could set the configuration key 
372    * {@link #CF_RENAME_PROP} yourself. The value should look like 
373    * <pre>srcCf1:destCf1,srcCf2:destCf2,....</pre>. This would have the same effect on
374    * the mapper behavior.
375    * 
376    * @param conf the Configuration in which the {@link #CF_RENAME_PROP} key will be
377    *  set
378    * @param renameMap a mapping from source CF names to destination CF names
379    */
380   static public void configureCfRenaming(Configuration conf, 
381           Map<String, String> renameMap) {
382     StringBuilder sb = new StringBuilder();
383     for(Map.Entry<String,String> entry: renameMap.entrySet()) {
384       String sourceCf = entry.getKey();
385       String destCf = entry.getValue();
386 
387       if(sourceCf.contains(":") || sourceCf.contains(",") || 
388               destCf.contains(":") || destCf.contains(",")) {
389         throw new IllegalArgumentException("Illegal character in CF names: " 
390               + sourceCf + ", " + destCf);
391       }
392 
393       if(sb.length() != 0) {
394         sb.append(",");
395       }
396       sb.append(sourceCf + ":" + destCf);
397     }
398     conf.set(CF_RENAME_PROP, sb.toString());
399   }
400 
401   /**
402    * Add a Filter to be instantiated on import
403    * @param conf Configuration to update (will be passed to the job)
404    * @param clazz {@link Filter} subclass to instantiate on the server.
405    * @param filterArgs List of arguments to pass to the filter on instantiation
406    */
407   public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz,
408       List<String> filterArgs) throws IOException {
409     conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName());
410     conf.setStrings(Import.FILTER_ARGS_CONF_KEY, filterArgs.toArray(new String[filterArgs.size()]));
411   }
412 
413   /**
414    * Sets up the actual job.
415    * @param conf The current configuration.
416    * @param args The command line parameters.
417    * @return The newly created job.
418    * @throws IOException When setting up the job fails.
419    */
420   public static Job createSubmittableJob(Configuration conf, String[] args)
421   throws IOException {
422     String tableName = args[0];
423     conf.set(TABLE_NAME, tableName);
424     Path inputDir = new Path(args[1]);
425     Job job = new Job(conf, NAME + "_" + tableName);
426     job.setJarByClass(Importer.class);
427     FileInputFormat.setInputPaths(job, inputDir);
428     job.setInputFormatClass(SequenceFileInputFormat.class);
429     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
430 
431     // make sure we get the filter in the jars
432     try {
433       Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
434       if (filter != null) {
435         TableMapReduceUtil.addDependencyJars(conf, filter);
436       }
437     } catch (Exception e) {
438       throw new IOException(e);
439     }
440 
441     if (hfileOutPath != null) {
442       job.setMapperClass(KeyValueImporter.class);
443       HTable table = new HTable(conf, tableName);
444       job.setReducerClass(KeyValueSortReducer.class);
445       Path outputDir = new Path(hfileOutPath);
446       FileOutputFormat.setOutputPath(job, outputDir);
447       job.setMapOutputKeyClass(ImmutableBytesWritable.class);
448       job.setMapOutputValueClass(KeyValue.class);
449       HFileOutputFormat.configureIncrementalLoad(job, table);
450       TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
451           com.google.common.base.Preconditions.class);
452     } else {
453       // No reducers.  Just write straight to table.  Call initTableReducerJob
454       // because it sets up the TableOutputFormat.
455       job.setMapperClass(Importer.class);
456       TableMapReduceUtil.initTableReducerJob(tableName, null, job);
457       job.setNumReduceTasks(0);
458     }
459     return job;
460   }
461 
462   /*
463    * @param errorMsg Error message.  Can be null.
464    */
465   private static void usage(final String errorMsg) {
466     if (errorMsg != null && errorMsg.length() > 0) {
467       System.err.println("ERROR: " + errorMsg);
468     }
469     System.err.println("Usage: Import [options] <tablename> <inputdir>");
470     System.err.println("By default Import will load data directly into HBase. To instead generate");
471     System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
472     System.err.println("  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
473     System.err
474         .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
475     System.err.println("  -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");
476     System.err.println("  -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");
477     System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "
478         + CF_RENAME_PROP + " property. Futher, filters will only use the"
479         + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "
480         + " whether the current row needs to be ignored completely for processing and "
481         + " Filter#filterKeyValue(KeyValue) method to determine if the KeyValue should be added;"
482         + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including"
483         + " the KeyValue.");
484     System.err.println("For performance consider the following options:\n"
485         + "  -Dmapred.map.tasks.speculative.execution=false\n"
486         + "  -Dmapred.reduce.tasks.speculative.execution=false\n"
487         + "  -D" + WAL_DURABILITY + "=<Used while writing data to hbase."
488             +" Allowed values are the supported durability values"
489             +" like SKIP_WAL/ASYNC_WAL/SYNC_WAL/...>");
490   }
491 
492   /**
493    * If the durability is set to {@link Durability#SKIP_WAL} and the data is imported to hbase, we
494    * need to flush all the regions of the table as the data is held in memory and is also not
495    * present in the Write Ahead Log to replay in scenarios of a crash. This method flushes all the
496    * regions of the table in the scenarios of import data to hbase with {@link Durability#SKIP_WAL}
497    */
498   public static void flushRegionsIfNecessary(Configuration conf) throws IOException,
499       InterruptedException {
500     String tableName = conf.get(TABLE_NAME);
501     HBaseAdmin hAdmin = null;
502     String durability = conf.get(WAL_DURABILITY);
503     // Need to flush if the data is written to hbase and skip wal is enabled.
504     if (conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null
505         && Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) {
506       try {
507         hAdmin = new HBaseAdmin(conf);
508         hAdmin.flush(tableName);
509       } finally {
510         if (hAdmin != null) {
511           hAdmin.close();
512         }
513       }
514     }
515   }
516 
517   /**
518    * Main entry point.
519    *
520    * @param args  The command line parameters.
521    * @throws Exception When running the job fails.
522    */
523   public static void main(String[] args) throws Exception {
524     Configuration conf = HBaseConfiguration.create();
525     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
526     if (otherArgs.length < 2) {
527       usage("Wrong number of arguments: " + otherArgs.length);
528       System.exit(-1);
529     }
530     String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER);
531     if (inputVersionString != null) {
532       conf.set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString);
533     }
534     Job job = createSubmittableJob(conf, otherArgs);
535     boolean isJobSuccessful = job.waitForCompletion(true);
536     if(isJobSuccessful){
537       // Flush all the regions of the table
538       flushRegionsIfNecessary(conf);
539     }
540     System.exit(job.waitForCompletion(true) ? 0 : 1);
541   }
542 }