View Javadoc

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