1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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
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
70 private static Filter filter;
71
72
73
74
75 static class KeyValueImporter
76 extends TableMapper<ImmutableBytesWritable, KeyValue> {
77 private Map<byte[], byte[]> cfRenameMap;
78
79
80
81
82
83
84
85
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
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
113
114 static class Importer
115 extends TableMapper<ImmutableBytesWritable, Mutation> {
116 private Map<byte[], byte[]> cfRenameMap;
117 private UUID clusterId;
118
119
120
121
122
123
124
125
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
145 if (kv == null) continue;
146
147 kv = convertKv(kv, cfRenameMap);
148
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
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
195
196
197
198
199
200 private static Filter instantiateFilter(Configuration conf) {
201
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
235
236 args.add(Bytes.toBytes("'" + arg + "'"));
237 }
238 return args;
239 }
240
241
242
243
244
245
246
247 private static KeyValue filterKv(KeyValue kv) throws IOException {
248
249 if (filter != null) {
250 Filter.ReturnCode code = filter.filterKeyValue(kv);
251 LOG.debug("Filter returned:" + code);
252
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
263 private static KeyValue convertKv(KeyValue kv, Map<byte[], byte[]> cfRenameMap) {
264 if(cfRenameMap != null) {
265
266 byte[] newCfName = cfRenameMap.get(kv.getFamily());
267 if(newCfName != null) {
268 kv = new KeyValue(kv.getBuffer(),
269 kv.getRowOffset(),
270 kv.getRowLength(),
271 newCfName,
272 0,
273 newCfName.length,
274 kv.getBuffer(),
275 kv.getQualifierOffset(),
276 kv.getQualifierLength(),
277 kv.getTimestamp(),
278 KeyValue.Type.codeToType(kv.getType()),
279 kv.getBuffer(),
280 kv.getValueOffset(),
281 kv.getValueLength());
282 }
283 }
284 return kv;
285 }
286
287
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
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
310
311
312
313
314
315
316
317
318
319
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
344
345
346
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
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
366
367
368
369
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
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
404
405 job.setMapperClass(Importer.class);
406 TableMapReduceUtil.initTableReducerJob(tableName, null, job);
407 job.setNumReduceTasks(0);
408 }
409 return job;
410 }
411
412
413
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
439
440
441
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 }