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