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