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