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.ByteArrayInputStream;
22 import java.io.DataInput;
23 import java.io.DataInputStream;
24 import java.io.DataOutput;
25 import java.io.IOException;
26 import java.lang.reflect.InvocationTargetException;
27 import java.lang.reflect.Method;
28 import java.util.ArrayList;
29 import java.util.Collections;
30 import java.util.List;
31 import java.util.Locale;
32 import java.util.Map;
33 import java.util.TreeMap;
34 import java.util.UUID;
35
36 import org.apache.commons.logging.Log;
37 import org.apache.commons.logging.LogFactory;
38 import org.apache.hadoop.conf.Configuration;
39 import org.apache.hadoop.fs.FileSystem;
40 import org.apache.hadoop.fs.Path;
41 import org.apache.hadoop.hbase.Cell;
42 import org.apache.hadoop.hbase.CellComparator;
43 import org.apache.hadoop.hbase.CellUtil;
44 import org.apache.hadoop.hbase.HBaseConfiguration;
45 import org.apache.hadoop.hbase.KeyValue;
46 import org.apache.hadoop.hbase.KeyValueUtil;
47 import org.apache.hadoop.hbase.TableName;
48 import org.apache.hadoop.hbase.Tag;
49 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
50 import org.apache.hadoop.hbase.classification.InterfaceAudience;
51 import org.apache.hadoop.hbase.classification.InterfaceStability;
52 import org.apache.hadoop.hbase.client.Connection;
53 import org.apache.hadoop.hbase.client.ConnectionFactory;
54 import org.apache.hadoop.hbase.client.Delete;
55 import org.apache.hadoop.hbase.client.Durability;
56 import org.apache.hadoop.hbase.client.HBaseAdmin;
57 import org.apache.hadoop.hbase.client.Mutation;
58 import org.apache.hadoop.hbase.client.Put;
59 import org.apache.hadoop.hbase.client.RegionLocator;
60 import org.apache.hadoop.hbase.client.Result;
61 import org.apache.hadoop.hbase.client.Table;
62 import org.apache.hadoop.hbase.filter.Filter;
63 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
64 import org.apache.hadoop.hbase.util.Bytes;
65 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
66 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
67 import org.apache.hadoop.io.RawComparator;
68 import org.apache.hadoop.io.WritableComparable;
69 import org.apache.hadoop.io.WritableComparator;
70 import org.apache.hadoop.mapreduce.Job;
71 import org.apache.hadoop.mapreduce.Partitioner;
72 import org.apache.hadoop.mapreduce.Reducer;
73 import org.apache.hadoop.mapreduce.TaskCounter;
74 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
75 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
76 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
77 import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
78 import org.apache.hadoop.util.GenericOptionsParser;
79 import org.apache.zookeeper.KeeperException;
80
81
82
83
84
85 @InterfaceAudience.Public
86 @InterfaceStability.Stable
87 public class Import {
88 private static final Log LOG = LogFactory.getLog(Import.class);
89 final static String NAME = "import";
90 public final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
91 public final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
92 public final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
93 public final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
94 public final static String TABLE_NAME = "import.table.name";
95 public final static String WAL_DURABILITY = "import.wal.durability";
96 public final static String HAS_LARGE_RESULT= "import.bulk.hasLargeResult";
97
98 public static class KeyValueWritableComparablePartitioner
99 extends Partitioner<KeyValueWritableComparable, KeyValue> {
100 private static KeyValueWritableComparable[] START_KEYS = null;
101 @Override
102 public int getPartition(KeyValueWritableComparable key, KeyValue value,
103 int numPartitions) {
104 for (int i = 0; i < START_KEYS.length; ++i) {
105 if (key.compareTo(START_KEYS[i]) <= 0) {
106 return i;
107 }
108 }
109 return START_KEYS.length;
110 }
111
112 }
113
114 public static class KeyValueWritableComparable
115 implements WritableComparable<KeyValueWritableComparable> {
116
117 private KeyValue kv = null;
118 private CellComparator cellComparator = new CellComparator();
119
120 static {
121
122 WritableComparator.define(KeyValueWritableComparable.class,
123 new KeyValueWritableComparator());
124 }
125
126 public KeyValueWritableComparable() {
127 }
128
129 public KeyValueWritableComparable(KeyValue kv) {
130 this.kv = kv;
131 }
132
133 @Override
134 public void write(DataOutput out) throws IOException {
135 KeyValue.write(kv, out);
136 }
137
138 @Override
139 public void readFields(DataInput in) throws IOException {
140 kv = KeyValue.create(in);
141 }
142
143 @Override
144 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
145 justification="This is wrong, yes, but we should be purging Writables, not fixing them")
146 public int compareTo(KeyValueWritableComparable o) {
147 return cellComparator.compare(this.kv, ((KeyValueWritableComparable)o).kv);
148 }
149
150 public static class KeyValueWritableComparator extends WritableComparator {
151
152 @Override
153 public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
154 try {
155 KeyValueWritableComparable kv1 = new KeyValueWritableComparable();
156 kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1)));
157 KeyValueWritableComparable kv2 = new KeyValueWritableComparable();
158 kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2)));
159 return compare(kv1, kv2);
160 } catch (IOException e) {
161 throw new RuntimeException(e);
162 }
163 }
164
165 }
166
167 }
168
169 public static class KeyValueReducer
170 extends
171 Reducer<KeyValueWritableComparable, KeyValue, ImmutableBytesWritable, KeyValue> {
172 protected void reduce(
173 KeyValueWritableComparable row,
174 Iterable<KeyValue> kvs,
175 Reducer<KeyValueWritableComparable,
176 KeyValue, ImmutableBytesWritable, KeyValue>.Context context)
177 throws java.io.IOException, InterruptedException {
178 int index = 0;
179 for (KeyValue kv : kvs) {
180 context.write(new ImmutableBytesWritable(kv.getRowArray()), kv);
181 if (++index % 100 == 0)
182 context.setStatus("Wrote " + index + " KeyValues, "
183 + "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
184 }
185 }
186 }
187
188 public static class KeyValueSortImporter
189 extends TableMapper<KeyValueWritableComparable, KeyValue> {
190 private Map<byte[], byte[]> cfRenameMap;
191 private Filter filter;
192 private static final Log LOG = LogFactory.getLog(KeyValueImporter.class);
193
194
195
196
197
198
199
200 @Override
201 public void map(ImmutableBytesWritable row, Result value, Context context)
202 throws IOException {
203 try {
204 if (LOG.isTraceEnabled()) {
205 LOG.trace("Considering the row."
206 + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
207 }
208 if (filter == null || !filter.filterRowKey(row.get(), row.getOffset(), row.getLength())) {
209 for (Cell kv : value.rawCells()) {
210 kv = filterKv(filter, kv);
211
212 if (kv == null) continue;
213
214 KeyValue ret = KeyValueUtil.ensureKeyValueTypeForMR(convertKv(kv, cfRenameMap));
215 context.write(new KeyValueWritableComparable(ret.createKeyOnly(false)), ret);
216 }
217 }
218 } catch (InterruptedException e) {
219 e.printStackTrace();
220 }
221 }
222
223 @Override
224 public void setup(Context context) throws IOException {
225 cfRenameMap = createCfRenameMap(context.getConfiguration());
226 filter = instantiateFilter(context.getConfiguration());
227 int reduceNum = context.getNumReduceTasks();
228 Configuration conf = context.getConfiguration();
229 TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME));
230 try (Connection conn = ConnectionFactory.createConnection(conf);
231 RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
232 byte[][] startKeys = regionLocator.getStartKeys();
233 if (startKeys.length != reduceNum) {
234 throw new IOException("Region split after job initialization");
235 }
236 KeyValueWritableComparable[] startKeyWraps =
237 new KeyValueWritableComparable[startKeys.length - 1];
238 for (int i = 1; i < startKeys.length; ++i) {
239 startKeyWraps[i - 1] =
240 new KeyValueWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i]));
241 }
242 KeyValueWritableComparablePartitioner.START_KEYS = startKeyWraps;
243 }
244 }
245 }
246
247
248
249
250
251 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
252 justification="Writables are going away and this has been this way forever")
253 public static class KeyValueImporter extends TableMapper<ImmutableBytesWritable, KeyValue> {
254 private Map<byte[], byte[]> cfRenameMap;
255 private Filter filter;
256 private static final Log LOG = LogFactory.getLog(KeyValueImporter.class);
257
258
259
260
261
262
263
264 @Override
265 public void map(ImmutableBytesWritable row, Result value,
266 Context context)
267 throws IOException {
268 try {
269 if (LOG.isTraceEnabled()) {
270 LOG.trace("Considering the row."
271 + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
272 }
273 if (filter == null || !filter.filterRowKey(row.get(), row.getOffset(), row.getLength())) {
274 for (Cell kv : value.rawCells()) {
275 kv = filterKv(filter, kv);
276
277 if (kv == null) continue;
278
279 context.write(row, KeyValueUtil.ensureKeyValueTypeForMR(convertKv(kv, cfRenameMap)));
280 }
281 }
282 } catch (InterruptedException e) {
283 e.printStackTrace();
284 }
285 }
286
287 @Override
288 public void setup(Context context) {
289 cfRenameMap = createCfRenameMap(context.getConfiguration());
290 filter = instantiateFilter(context.getConfiguration());
291 }
292 }
293
294
295
296
297 public static class Importer extends TableMapper<ImmutableBytesWritable, Mutation> {
298 private Map<byte[], byte[]> cfRenameMap;
299 private List<UUID> clusterIds;
300 private Filter filter;
301 private Durability durability;
302
303
304
305
306
307
308
309 @Override
310 public void map(ImmutableBytesWritable row, Result value,
311 Context context)
312 throws IOException {
313 try {
314 writeResult(row, value, context);
315 } catch (InterruptedException e) {
316 e.printStackTrace();
317 }
318 }
319
320 private void writeResult(ImmutableBytesWritable key, Result result, Context context)
321 throws IOException, InterruptedException {
322 Put put = null;
323 Delete delete = null;
324 if (LOG.isTraceEnabled()) {
325 LOG.trace("Considering the row."
326 + Bytes.toString(key.get(), key.getOffset(), key.getLength()));
327 }
328 if (filter == null || !filter.filterRowKey(key.get(), key.getOffset(), key.getLength())) {
329 processKV(key, result, context, put, delete);
330 }
331 }
332
333 protected void processKV(ImmutableBytesWritable key, Result result, Context context, Put put,
334 Delete delete) throws IOException, InterruptedException {
335 for (Cell kv : result.rawCells()) {
336 kv = filterKv(filter, kv);
337
338 if (kv == null) continue;
339
340 kv = convertKv(kv, cfRenameMap);
341
342
343
344
345
346
347
348
349 if (CellUtil.isDeleteFamily(kv)) {
350 Delete deleteFamily = new Delete(key.get());
351 deleteFamily.addDeleteMarker(kv);
352 if (durability != null) {
353 deleteFamily.setDurability(durability);
354 }
355 deleteFamily.setClusterIds(clusterIds);
356 context.write(key, deleteFamily);
357 } else if (CellUtil.isDelete(kv)) {
358 if (delete == null) {
359 delete = new Delete(key.get());
360 }
361 delete.addDeleteMarker(kv);
362 } else {
363 if (put == null) {
364 put = new Put(key.get());
365 }
366 addPutToKv(put, kv);
367 }
368 }
369 if (put != null) {
370 if (durability != null) {
371 put.setDurability(durability);
372 }
373 put.setClusterIds(clusterIds);
374 context.write(key, put);
375 }
376 if (delete != null) {
377 if (durability != null) {
378 delete.setDurability(durability);
379 }
380 delete.setClusterIds(clusterIds);
381 context.write(key, delete);
382 }
383 }
384
385 protected void addPutToKv(Put put, Cell kv) throws IOException {
386 put.add(kv);
387 }
388
389 @Override
390 public void setup(Context context) {
391 Configuration conf = context.getConfiguration();
392 cfRenameMap = createCfRenameMap(conf);
393 filter = instantiateFilter(conf);
394 String durabilityStr = conf.get(WAL_DURABILITY);
395 if(durabilityStr != null){
396 durability = Durability.valueOf(durabilityStr.toUpperCase(Locale.ROOT));
397 }
398
399 ZooKeeperWatcher zkw = null;
400 Exception ex = null;
401 try {
402 zkw = new ZooKeeperWatcher(conf, context.getTaskAttemptID().toString(), null);
403 clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
404 } catch (ZooKeeperConnectionException e) {
405 ex = e;
406 LOG.error("Problem connecting to ZooKeper during task setup", e);
407 } catch (KeeperException e) {
408 ex = e;
409 LOG.error("Problem reading ZooKeeper data during task setup", e);
410 } catch (IOException e) {
411 ex = e;
412 LOG.error("Problem setting up task", e);
413 } finally {
414 if (zkw != null) zkw.close();
415 }
416 if (clusterIds == null) {
417
418 throw new RuntimeException(ex);
419 }
420 }
421 }
422
423
424
425
426
427
428
429
430 public static Filter instantiateFilter(Configuration conf) {
431
432 Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
433 if (filterClass == null) {
434 LOG.debug("No configured filter class, accepting all keyvalues.");
435 return null;
436 }
437 LOG.debug("Attempting to create filter:" + filterClass);
438 String[] filterArgs = conf.getStrings(FILTER_ARGS_CONF_KEY);
439 ArrayList<byte[]> quotedArgs = toQuotedByteArrays(filterArgs);
440 try {
441 Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
442 return (Filter) m.invoke(null, quotedArgs);
443 } catch (IllegalAccessException e) {
444 LOG.error("Couldn't instantiate filter!", e);
445 throw new RuntimeException(e);
446 } catch (SecurityException e) {
447 LOG.error("Couldn't instantiate filter!", e);
448 throw new RuntimeException(e);
449 } catch (NoSuchMethodException e) {
450 LOG.error("Couldn't instantiate filter!", e);
451 throw new RuntimeException(e);
452 } catch (IllegalArgumentException e) {
453 LOG.error("Couldn't instantiate filter!", e);
454 throw new RuntimeException(e);
455 } catch (InvocationTargetException e) {
456 LOG.error("Couldn't instantiate filter!", e);
457 throw new RuntimeException(e);
458 }
459 }
460
461 private static ArrayList<byte[]> toQuotedByteArrays(String... stringArgs) {
462 ArrayList<byte[]> quotedArgs = new ArrayList<byte[]>();
463 for (String stringArg : stringArgs) {
464
465
466 quotedArgs.add(Bytes.toBytes("'" + stringArg + "'"));
467 }
468 return quotedArgs;
469 }
470
471
472
473
474
475
476
477 public static Cell filterKv(Filter filter, Cell kv) throws IOException {
478
479 if (filter != null) {
480 Filter.ReturnCode code = filter.filterKeyValue(kv);
481 if (LOG.isTraceEnabled()) {
482 LOG.trace("Filter returned:" + code + " for the key value:" + kv);
483 }
484
485 if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
486 .equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
487 return null;
488 }
489 }
490 return kv;
491 }
492
493
494 private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {
495 if(cfRenameMap != null) {
496
497 byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
498 if(newCfName != null) {
499 List<Tag> tags = null;
500 if (kv.getTagsLength() > 0) {
501 tags = Tag.asList(kv.getTagsArray(), kv.getTagsOffset(), kv.getTagsLength());
502 }
503 kv = new KeyValue(kv.getRowArray(),
504 kv.getRowOffset(),
505 kv.getRowLength(),
506 newCfName,
507 0,
508 newCfName.length,
509 kv.getQualifierArray(),
510 kv.getQualifierOffset(),
511 kv.getQualifierLength(),
512 kv.getTimestamp(),
513 KeyValue.Type.codeToType(kv.getTypeByte()),
514 kv.getValueArray(),
515 kv.getValueOffset(),
516 kv.getValueLength(),
517 tags);
518 }
519 }
520 return kv;
521 }
522
523
524 private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) {
525 Map<byte[], byte[]> cfRenameMap = null;
526 String allMappingsPropVal = conf.get(CF_RENAME_PROP);
527 if(allMappingsPropVal != null) {
528
529 String[] allMappings = allMappingsPropVal.split(",");
530 for (String mapping: allMappings) {
531 if(cfRenameMap == null) {
532 cfRenameMap = new TreeMap<byte[],byte[]>(Bytes.BYTES_COMPARATOR);
533 }
534 String [] srcAndDest = mapping.split(":");
535 if(srcAndDest.length != 2) {
536 continue;
537 }
538 cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
539 }
540 }
541 return cfRenameMap;
542 }
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557 static public void configureCfRenaming(Configuration conf,
558 Map<String, String> renameMap) {
559 StringBuilder sb = new StringBuilder();
560 for(Map.Entry<String,String> entry: renameMap.entrySet()) {
561 String sourceCf = entry.getKey();
562 String destCf = entry.getValue();
563
564 if(sourceCf.contains(":") || sourceCf.contains(",") ||
565 destCf.contains(":") || destCf.contains(",")) {
566 throw new IllegalArgumentException("Illegal character in CF names: "
567 + sourceCf + ", " + destCf);
568 }
569
570 if(sb.length() != 0) {
571 sb.append(",");
572 }
573 sb.append(sourceCf + ":" + destCf);
574 }
575 conf.set(CF_RENAME_PROP, sb.toString());
576 }
577
578
579
580
581
582
583
584 public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz,
585 List<String> filterArgs) throws IOException {
586 conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName());
587 conf.setStrings(Import.FILTER_ARGS_CONF_KEY, filterArgs.toArray(new String[filterArgs.size()]));
588 }
589
590
591
592
593
594
595
596
597 public static Job createSubmittableJob(Configuration conf, String[] args)
598 throws IOException {
599 TableName tableName = TableName.valueOf(args[0]);
600 conf.set(TABLE_NAME, tableName.getNameAsString());
601 Path inputDir = new Path(args[1]);
602 Job job = new Job(conf, NAME + "_" + tableName);
603 job.setJarByClass(Importer.class);
604 FileInputFormat.setInputPaths(job, inputDir);
605 job.setInputFormatClass(SequenceFileInputFormat.class);
606 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
607
608
609 try {
610 Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
611 if (filter != null) {
612 TableMapReduceUtil.addDependencyJarsForClasses(conf, filter);
613 }
614 } catch (Exception e) {
615 throw new IOException(e);
616 }
617
618 if (hfileOutPath != null && conf.getBoolean(HAS_LARGE_RESULT, false)) {
619 LOG.info("Use Large Result!!");
620 try (Connection conn = ConnectionFactory.createConnection(conf);
621 Table table = conn.getTable(tableName);
622 RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
623 HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
624 job.setMapperClass(KeyValueSortImporter.class);
625 job.setReducerClass(KeyValueReducer.class);
626 Path outputDir = new Path(hfileOutPath);
627 FileOutputFormat.setOutputPath(job, outputDir);
628 job.setMapOutputKeyClass(KeyValueWritableComparable.class);
629 job.setMapOutputValueClass(KeyValue.class);
630 job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
631 KeyValueWritableComparable.KeyValueWritableComparator.class,
632 RawComparator.class);
633 Path partitionsPath =
634 new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration()));
635 FileSystem fs = FileSystem.get(job.getConfiguration());
636 fs.deleteOnExit(partitionsPath);
637 job.setPartitionerClass(KeyValueWritableComparablePartitioner.class);
638 job.setNumReduceTasks(regionLocator.getStartKeys().length);
639 TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
640 com.google.common.base.Preconditions.class);
641 }
642 } else if (hfileOutPath != null) {
643 job.setMapperClass(KeyValueImporter.class);
644 try (Connection conn = ConnectionFactory.createConnection(conf);
645 Table table = conn.getTable(tableName);
646 RegionLocator regionLocator = conn.getRegionLocator(tableName)){
647 job.setReducerClass(KeyValueSortReducer.class);
648 Path outputDir = new Path(hfileOutPath);
649 FileOutputFormat.setOutputPath(job, outputDir);
650 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
651 job.setMapOutputValueClass(KeyValue.class);
652 HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
653 TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
654 com.google.common.base.Preconditions.class);
655 }
656 } else {
657
658
659 job.setMapperClass(Importer.class);
660 TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job);
661 job.setNumReduceTasks(0);
662 }
663 return job;
664 }
665
666
667
668
669 private static void usage(final String errorMsg) {
670 if (errorMsg != null && errorMsg.length() > 0) {
671 System.err.println("ERROR: " + errorMsg);
672 }
673 System.err.println("Usage: Import [options] <tablename> <inputdir>");
674 System.err.println("By default Import will load data directly into HBase. To instead generate");
675 System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
676 System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
677 System.err.println("If there is a large result that includes too much KeyValue "
678 + "whitch can occur OOME caused by the memery sort in reducer, pass the option:");
679 System.err.println(" -D" + HAS_LARGE_RESULT + "=true");
680 System.err
681 .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
682 System.err.println(" -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");
683 System.err.println(" -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");
684 System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "
685 + CF_RENAME_PROP + " property. Futher, filters will only use the"
686 + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "
687 + " whether the current row needs to be ignored completely for processing and "
688 + " Filter#filterKeyValue(KeyValue) method to determine if the KeyValue should be added;"
689 + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including"
690 + " the KeyValue.");
691 System.err.println("To import data exported from HBase 0.94, use");
692 System.err.println(" -Dhbase.import.version=0.94");
693 System.err.println("For performance consider the following options:\n"
694 + " -Dmapreduce.map.speculative=false\n"
695 + " -Dmapreduce.reduce.speculative=false\n"
696 + " -D" + WAL_DURABILITY + "=<Used while writing data to hbase."
697 +" Allowed values are the supported durability values"
698 +" like SKIP_WAL/ASYNC_WAL/SYNC_WAL/...>");
699 }
700
701
702
703
704
705
706
707 public static void flushRegionsIfNecessary(Configuration conf) throws IOException,
708 InterruptedException {
709 String tableName = conf.get(TABLE_NAME);
710 HBaseAdmin hAdmin = null;
711 String durability = conf.get(WAL_DURABILITY);
712
713 if (conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null
714 && Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) {
715 try {
716 hAdmin = new HBaseAdmin(conf);
717 hAdmin.flush(tableName);
718 } finally {
719 if (hAdmin != null) {
720 hAdmin.close();
721 }
722 }
723 }
724 }
725
726
727
728
729
730
731
732 public static void main(String[] args) throws Exception {
733 Configuration conf = HBaseConfiguration.create();
734 String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
735 if (otherArgs.length < 2) {
736 usage("Wrong number of arguments: " + otherArgs.length);
737 System.exit(-1);
738 }
739 String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER);
740 if (inputVersionString != null) {
741 conf.set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString);
742 }
743 Job job = createSubmittableJob(conf, otherArgs);
744 boolean isJobSuccessful = job.waitForCompletion(true);
745 if(isJobSuccessful){
746
747 flushRegionsIfNecessary(conf);
748 }
749 long inputRecords = job.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue();
750 long outputRecords = job.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS).getValue();
751 if (outputRecords < inputRecords) {
752 System.err.println("Warning, not all records were imported (maybe filtered out).");
753 if (outputRecords == 0) {
754 System.err.println("If the data was exported from HBase 0.94 "+
755 "consider using -Dhbase.import.version=0.94.");
756 }
757 }
758
759 System.exit(isJobSuccessful ? 0 : 1);
760 }
761 }