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