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.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.hbase.classification.InterfaceAudience;
34 import org.apache.hadoop.hbase.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.TableName;
43 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
44 import org.apache.hadoop.hbase.client.Delete;
45 import org.apache.hadoop.hbase.client.Durability;
46 import org.apache.hadoop.hbase.client.HBaseAdmin;
47 import org.apache.hadoop.hbase.client.HTable;
48 import org.apache.hadoop.hbase.client.Mutation;
49 import org.apache.hadoop.hbase.client.Put;
50 import org.apache.hadoop.hbase.client.Result;
51 import org.apache.hadoop.hbase.filter.Filter;
52 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
53 import org.apache.hadoop.hbase.util.Bytes;
54 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
55 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
56 import org.apache.hadoop.mapreduce.Job;
57 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
58 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
59 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
60 import org.apache.hadoop.util.GenericOptionsParser;
61 import org.apache.zookeeper.KeeperException;
62
63
64
65
66
67 @InterfaceAudience.Public
68 @InterfaceStability.Stable
69 public class Import {
70 private static final Log LOG = LogFactory.getLog(Import.class);
71 final static String NAME = "import";
72 public final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
73 public final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
74 public final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
75 public final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
76 public final static String TABLE_NAME = "import.table.name";
77 public final static String WAL_DURABILITY = "import.wal.durability";
78
79
80
81
82 public static class KeyValueImporter extends TableMapper<ImmutableBytesWritable, KeyValue> {
83 private Map<byte[], byte[]> cfRenameMap;
84 private Filter filter;
85 private static final Log LOG = LogFactory.getLog(KeyValueImporter.class);
86
87
88
89
90
91
92
93 @Override
94 public void map(ImmutableBytesWritable row, Result value,
95 Context context)
96 throws IOException {
97 try {
98 if (LOG.isTraceEnabled()) {
99 LOG.trace("Considering the row."
100 + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
101 }
102 if (filter == null || !filter.filterRowKey(row.get(), row.getOffset(), row.getLength())) {
103 for (Cell kv : value.rawCells()) {
104 kv = filterKv(filter, kv);
105
106 if (kv == null) continue;
107
108 context.write(row, KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap)));
109 }
110 }
111 } catch (InterruptedException e) {
112 e.printStackTrace();
113 }
114 }
115
116 @Override
117 public void setup(Context context) {
118 cfRenameMap = createCfRenameMap(context.getConfiguration());
119 filter = instantiateFilter(context.getConfiguration());
120 }
121 }
122
123
124
125
126 public static class Importer extends TableMapper<ImmutableBytesWritable, Mutation> {
127 private Map<byte[], byte[]> cfRenameMap;
128 private List<UUID> clusterIds;
129 private Filter filter;
130 private Durability durability;
131
132
133
134
135
136
137
138 @Override
139 public void map(ImmutableBytesWritable row, Result value,
140 Context context)
141 throws IOException {
142 try {
143 writeResult(row, value, context);
144 } catch (InterruptedException e) {
145 e.printStackTrace();
146 }
147 }
148
149 private void writeResult(ImmutableBytesWritable key, Result result, Context context)
150 throws IOException, InterruptedException {
151 Put put = null;
152 Delete delete = null;
153 if (LOG.isTraceEnabled()) {
154 LOG.trace("Considering the row."
155 + Bytes.toString(key.get(), key.getOffset(), key.getLength()));
156 }
157 if (filter == null || !filter.filterRowKey(key.get(), key.getOffset(), key.getLength())) {
158 processKV(key, result, context, put, delete);
159 }
160 }
161
162 protected void processKV(ImmutableBytesWritable key, Result result, Context context, Put put,
163 Delete delete) throws IOException, InterruptedException {
164 for (Cell kv : result.rawCells()) {
165 kv = filterKv(filter, kv);
166
167 if (kv == null) continue;
168
169 kv = convertKv(kv, cfRenameMap);
170
171
172
173
174
175
176
177
178 if (CellUtil.isDeleteFamily(kv)) {
179 Delete deleteFamily = new Delete(key.get());
180 deleteFamily.addDeleteMarker(kv);
181 if (durability != null) {
182 deleteFamily.setDurability(durability);
183 }
184 deleteFamily.setClusterIds(clusterIds);
185 context.write(key, deleteFamily);
186 } else if (CellUtil.isDelete(kv)) {
187 if (delete == null) {
188 delete = new Delete(key.get());
189 }
190 delete.addDeleteMarker(kv);
191 } else {
192 if (put == null) {
193 put = new Put(key.get());
194 }
195 addPutToKv(put, kv);
196 }
197 }
198 if (put != null) {
199 if (durability != null) {
200 put.setDurability(durability);
201 }
202 put.setClusterIds(clusterIds);
203 context.write(key, put);
204 }
205 if (delete != null) {
206 if (durability != null) {
207 delete.setDurability(durability);
208 }
209 delete.setClusterIds(clusterIds);
210 context.write(key, delete);
211 }
212 }
213
214 protected void addPutToKv(Put put, Cell kv) throws IOException {
215 put.add(kv);
216 }
217
218 @Override
219 public void setup(Context context) {
220 Configuration conf = context.getConfiguration();
221 cfRenameMap = createCfRenameMap(conf);
222 filter = instantiateFilter(conf);
223 String durabilityStr = conf.get(WAL_DURABILITY);
224 if(durabilityStr != null){
225 durability = Durability.valueOf(durabilityStr.toUpperCase());
226 }
227
228 ZooKeeperWatcher zkw = null;
229 Exception ex = null;
230 try {
231 zkw = new ZooKeeperWatcher(conf, context.getTaskAttemptID().toString(), null);
232 clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
233 } catch (ZooKeeperConnectionException e) {
234 ex = e;
235 LOG.error("Problem connecting to ZooKeper during task setup", e);
236 } catch (KeeperException e) {
237 ex = e;
238 LOG.error("Problem reading ZooKeeper data during task setup", e);
239 } catch (IOException e) {
240 ex = e;
241 LOG.error("Problem setting up task", e);
242 } finally {
243 if (zkw != null) zkw.close();
244 }
245 if (clusterIds == null) {
246
247 throw new RuntimeException(ex);
248 }
249 }
250 }
251
252
253
254
255
256
257
258
259 public static Filter instantiateFilter(Configuration conf) {
260
261 Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
262 if (filterClass == null) {
263 LOG.debug("No configured filter class, accepting all keyvalues.");
264 return null;
265 }
266 LOG.debug("Attempting to create filter:" + filterClass);
267 String[] filterArgs = conf.getStrings(FILTER_ARGS_CONF_KEY);
268 ArrayList<byte[]> quotedArgs = toQuotedByteArrays(filterArgs);
269 try {
270 Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
271 return (Filter) m.invoke(null, quotedArgs);
272 } catch (IllegalAccessException e) {
273 LOG.error("Couldn't instantiate filter!", e);
274 throw new RuntimeException(e);
275 } catch (SecurityException e) {
276 LOG.error("Couldn't instantiate filter!", e);
277 throw new RuntimeException(e);
278 } catch (NoSuchMethodException e) {
279 LOG.error("Couldn't instantiate filter!", e);
280 throw new RuntimeException(e);
281 } catch (IllegalArgumentException e) {
282 LOG.error("Couldn't instantiate filter!", e);
283 throw new RuntimeException(e);
284 } catch (InvocationTargetException e) {
285 LOG.error("Couldn't instantiate filter!", e);
286 throw new RuntimeException(e);
287 }
288 }
289
290 private static ArrayList<byte[]> toQuotedByteArrays(String... stringArgs) {
291 ArrayList<byte[]> quotedArgs = new ArrayList<byte[]>();
292 for (String stringArg : stringArgs) {
293
294
295 quotedArgs.add(Bytes.toBytes("'" + stringArg + "'"));
296 }
297 return quotedArgs;
298 }
299
300
301
302
303
304
305
306 public static Cell filterKv(Filter filter, Cell kv) throws IOException {
307
308 if (filter != null) {
309 Filter.ReturnCode code = filter.filterKeyValue(kv);
310 if (LOG.isTraceEnabled()) {
311 LOG.trace("Filter returned:" + code + " for the key value:" + kv);
312 }
313
314 if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
315 .equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
316 return null;
317 }
318 }
319 return kv;
320 }
321
322
323 private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {
324 if(cfRenameMap != null) {
325
326 byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
327 if(newCfName != null) {
328 kv = new KeyValue(kv.getRowArray(),
329 kv.getRowOffset(),
330 kv.getRowLength(),
331 newCfName,
332 0,
333 newCfName.length,
334 kv.getQualifierArray(),
335 kv.getQualifierOffset(),
336 kv.getQualifierLength(),
337 kv.getTimestamp(),
338 KeyValue.Type.codeToType(kv.getTypeByte()),
339 kv.getValueArray(),
340 kv.getValueOffset(),
341 kv.getValueLength());
342 }
343 }
344 return kv;
345 }
346
347
348 private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) {
349 Map<byte[], byte[]> cfRenameMap = null;
350 String allMappingsPropVal = conf.get(CF_RENAME_PROP);
351 if(allMappingsPropVal != null) {
352
353 String[] allMappings = allMappingsPropVal.split(",");
354 for (String mapping: allMappings) {
355 if(cfRenameMap == null) {
356 cfRenameMap = new TreeMap<byte[],byte[]>(Bytes.BYTES_COMPARATOR);
357 }
358 String [] srcAndDest = mapping.split(":");
359 if(srcAndDest.length != 2) {
360 continue;
361 }
362 cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
363 }
364 }
365 return cfRenameMap;
366 }
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381 static public void configureCfRenaming(Configuration conf,
382 Map<String, String> renameMap) {
383 StringBuilder sb = new StringBuilder();
384 for(Map.Entry<String,String> entry: renameMap.entrySet()) {
385 String sourceCf = entry.getKey();
386 String destCf = entry.getValue();
387
388 if(sourceCf.contains(":") || sourceCf.contains(",") ||
389 destCf.contains(":") || destCf.contains(",")) {
390 throw new IllegalArgumentException("Illegal character in CF names: "
391 + sourceCf + ", " + destCf);
392 }
393
394 if(sb.length() != 0) {
395 sb.append(",");
396 }
397 sb.append(sourceCf + ":" + destCf);
398 }
399 conf.set(CF_RENAME_PROP, sb.toString());
400 }
401
402
403
404
405
406
407
408 public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz,
409 List<String> filterArgs) throws IOException {
410 conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName());
411 conf.setStrings(Import.FILTER_ARGS_CONF_KEY, filterArgs.toArray(new String[filterArgs.size()]));
412 }
413
414
415
416
417
418
419
420
421 public static Job createSubmittableJob(Configuration conf, String[] args)
422 throws IOException {
423 TableName tableName = TableName.valueOf(args[0]);
424 conf.set(TABLE_NAME, tableName.getNameAsString());
425 Path inputDir = new Path(args[1]);
426 Job job = new Job(conf, NAME + "_" + tableName);
427 job.setJarByClass(Importer.class);
428 FileInputFormat.setInputPaths(job, inputDir);
429 job.setInputFormatClass(SequenceFileInputFormat.class);
430 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
431
432
433 try {
434 Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
435 if (filter != null) {
436 TableMapReduceUtil.addDependencyJars(conf, filter);
437 }
438 } catch (Exception e) {
439 throw new IOException(e);
440 }
441
442 if (hfileOutPath != null) {
443 job.setMapperClass(KeyValueImporter.class);
444 HTable table = new HTable(conf, tableName);
445 job.setReducerClass(KeyValueSortReducer.class);
446 Path outputDir = new Path(hfileOutPath);
447 FileOutputFormat.setOutputPath(job, outputDir);
448 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
449 job.setMapOutputValueClass(KeyValue.class);
450 HFileOutputFormat2.configureIncrementalLoad(job, table, table);
451 TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
452 com.google.common.base.Preconditions.class);
453 } else {
454
455
456 job.setMapperClass(Importer.class);
457 TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job);
458 job.setNumReduceTasks(0);
459 }
460 return job;
461 }
462
463
464
465
466 private static void usage(final String errorMsg) {
467 if (errorMsg != null && errorMsg.length() > 0) {
468 System.err.println("ERROR: " + errorMsg);
469 }
470 System.err.println("Usage: Import [options] <tablename> <inputdir>");
471 System.err.println("By default Import will load data directly into HBase. To instead generate");
472 System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
473 System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
474 System.err
475 .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
476 System.err.println(" -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");
477 System.err.println(" -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");
478 System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "
479 + CF_RENAME_PROP + " property. Futher, filters will only use the"
480 + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "
481 + " whether the current row needs to be ignored completely for processing and "
482 + " Filter#filterKeyValue(KeyValue) method to determine if the KeyValue should be added;"
483 + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including"
484 + " the KeyValue.");
485 System.err.println("For performance consider the following options:\n"
486 + " -Dmapreduce.map.speculative=false\n"
487 + " -Dmapreduce.reduce.speculative=false\n"
488 + " -D" + WAL_DURABILITY + "=<Used while writing data to hbase."
489 +" Allowed values are the supported durability values"
490 +" like SKIP_WAL/ASYNC_WAL/SYNC_WAL/...>");
491 }
492
493
494
495
496
497
498
499 public static void flushRegionsIfNecessary(Configuration conf) throws IOException,
500 InterruptedException {
501 String tableName = conf.get(TABLE_NAME);
502 HBaseAdmin hAdmin = null;
503 String durability = conf.get(WAL_DURABILITY);
504
505 if (conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null
506 && Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) {
507 try {
508 hAdmin = new HBaseAdmin(conf);
509 hAdmin.flush(tableName);
510 } finally {
511 if (hAdmin != null) {
512 hAdmin.close();
513 }
514 }
515 }
516 }
517
518
519
520
521
522
523
524 public static void main(String[] args) throws Exception {
525 Configuration conf = HBaseConfiguration.create();
526 String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
527 if (otherArgs.length < 2) {
528 usage("Wrong number of arguments: " + otherArgs.length);
529 System.exit(-1);
530 }
531 String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER);
532 if (inputVersionString != null) {
533 conf.set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString);
534 }
535 Job job = createSubmittableJob(conf, otherArgs);
536 boolean isJobSuccessful = job.waitForCompletion(true);
537 if(isJobSuccessful){
538
539 flushRegionsIfNecessary(conf);
540 }
541 System.exit(job.waitForCompletion(true) ? 0 : 1);
542 }
543 }