/
Import.java
984 lines (912 loc) · 38.6 KB
/
Import.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.mapreduce;
import java.io.ByteArrayInputStream;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.TreeMap;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Import data written by {@link Export}.
*/
@InterfaceAudience.Public
public class Import extends Configured implements Tool {
private static final Logger LOG = LoggerFactory.getLogger(Import.class);
final static String NAME = "import";
public final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
public final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
public final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
public final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
public final static String TABLE_NAME = "import.table.name";
public final static String WAL_DURABILITY = "import.wal.durability";
public final static String HAS_LARGE_RESULT= "import.bulk.hasLargeResult";
private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
public static class CellWritableComparablePartitioner
extends Partitioner<CellWritableComparable, Cell> {
private static CellWritableComparable[] START_KEYS = null;
@Override
public int getPartition(CellWritableComparable key, Cell value,
int numPartitions) {
for (int i = 0; i < START_KEYS.length; ++i) {
if (key.compareTo(START_KEYS[i]) <= 0) {
return i;
}
}
return START_KEYS.length;
}
}
/**
* @deprecated Use {@link CellWritableComparablePartitioner}. Will be removed
* from 3.0 onwards
*/
@Deprecated
public static class KeyValueWritableComparablePartitioner
extends Partitioner<KeyValueWritableComparable, KeyValue> {
private static KeyValueWritableComparable[] START_KEYS = null;
@Override
public int getPartition(KeyValueWritableComparable key, KeyValue value, int numPartitions) {
for (int i = 0; i < START_KEYS.length; ++i) {
if (key.compareTo(START_KEYS[i]) <= 0) {
return i;
}
}
return START_KEYS.length;
}
}
public static class KeyValueWritableComparable
implements WritableComparable<KeyValueWritableComparable> {
private KeyValue kv = null;
static {
// register this comparator
WritableComparator.define(KeyValueWritableComparable.class, new KeyValueWritableComparator());
}
public KeyValueWritableComparable() {
}
public KeyValueWritableComparable(KeyValue kv) {
this.kv = kv;
}
@Override
public void write(DataOutput out) throws IOException {
KeyValue.write(kv, out);
}
@Override
public void readFields(DataInput in) throws IOException {
kv = KeyValue.create(in);
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS",
justification = "This is wrong, yes, but we should be purging Writables, not fixing them")
public int compareTo(KeyValueWritableComparable o) {
return CellComparator.getInstance().compare(this.kv, o.kv);
}
public static class KeyValueWritableComparator extends WritableComparator {
@Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
try {
KeyValueWritableComparable kv1 = new KeyValueWritableComparable();
kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1)));
KeyValueWritableComparable kv2 = new KeyValueWritableComparable();
kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2)));
return compare(kv1, kv2);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
}
public static class CellWritableComparable
implements WritableComparable<CellWritableComparable> {
private Cell kv = null;
static {
// register this comparator
WritableComparator.define(CellWritableComparable.class,
new CellWritableComparator());
}
public CellWritableComparable() {
}
public CellWritableComparable(Cell kv) {
this.kv = kv;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(PrivateCellUtil.estimatedSerializedSizeOfKey(kv));
out.writeInt(0);
PrivateCellUtil.writeFlatKey(kv, out);
}
@Override
public void readFields(DataInput in) throws IOException {
kv = KeyValue.create(in);
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS",
justification = "This is wrong, yes, but we should be purging Writables, not fixing them")
public int compareTo(CellWritableComparable o) {
return CellComparator.getInstance().compare(this.kv, o.kv);
}
public static class CellWritableComparator extends WritableComparator {
@Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
try {
CellWritableComparable kv1 = new CellWritableComparable();
kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1)));
CellWritableComparable kv2 = new CellWritableComparable();
kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2)));
return compare(kv1, kv2);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
}
/**
* @deprecated Use {@link CellReducer}. Will be removed from 3.0 onwards
*/
@Deprecated
public static class KeyValueReducer
extends Reducer<KeyValueWritableComparable, KeyValue, ImmutableBytesWritable, KeyValue> {
protected void reduce(KeyValueWritableComparable row, Iterable<KeyValue> kvs,
Reducer<KeyValueWritableComparable, KeyValue, ImmutableBytesWritable, KeyValue>.Context context)
throws java.io.IOException, InterruptedException {
int index = 0;
for (KeyValue kv : kvs) {
context.write(new ImmutableBytesWritable(kv.getRowArray()), kv);
if (++index % 100 == 0) context.setStatus("Wrote " + index + " KeyValues, "
+ "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
}
}
}
public static class CellReducer
extends
Reducer<CellWritableComparable, Cell, ImmutableBytesWritable, Cell> {
protected void reduce(
CellWritableComparable row,
Iterable<Cell> kvs,
Reducer<CellWritableComparable,
Cell, ImmutableBytesWritable, Cell>.Context context)
throws java.io.IOException, InterruptedException {
int index = 0;
for (Cell kv : kvs) {
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)),
new MapReduceExtendedCell(kv));
if (++index % 100 == 0)
context.setStatus("Wrote " + index + " KeyValues, "
+ "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
}
}
}
/**
* @deprecated Use {@link CellSortImporter}. Will be removed from 3.0 onwards
*/
@Deprecated
public static class KeyValueSortImporter
extends TableMapper<KeyValueWritableComparable, KeyValue> {
private Map<byte[], byte[]> cfRenameMap;
private Filter filter;
private static final Logger LOG = LoggerFactory.getLogger(KeyValueSortImporter.class);
/**
* @param row The current table row key.
* @param value The columns.
* @param context The current context.
* @throws IOException When something is broken with the data.
*/
@Override
public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException {
try {
if (LOG.isTraceEnabled()) {
LOG.trace(
"Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
}
if (filter == null || !filter.filterRowKey(
PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))) {
for (Cell kv : value.rawCells()) {
kv = filterKv(filter, kv);
// skip if we filtered it out
if (kv == null) continue;
// TODO get rid of ensureKeyValue
KeyValue ret = KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap));
context.write(new KeyValueWritableComparable(ret.createKeyOnly(false)), ret);
}
}
} catch (InterruptedException e) {
e.printStackTrace();
}
}
@Override
public void setup(Context context) throws IOException {
cfRenameMap = createCfRenameMap(context.getConfiguration());
filter = instantiateFilter(context.getConfiguration());
int reduceNum = context.getNumReduceTasks();
Configuration conf = context.getConfiguration();
TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME));
try (Connection conn = ConnectionFactory.createConnection(conf);
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
byte[][] startKeys = regionLocator.getStartKeys();
if (startKeys.length != reduceNum) {
throw new IOException("Region split after job initialization");
}
KeyValueWritableComparable[] startKeyWraps =
new KeyValueWritableComparable[startKeys.length - 1];
for (int i = 1; i < startKeys.length; ++i) {
startKeyWraps[i - 1] =
new KeyValueWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i]));
}
KeyValueWritableComparablePartitioner.START_KEYS = startKeyWraps;
}
}
}
/**
* A mapper that just writes out KeyValues.
* @deprecated Use {@link CellImporter}. Will be removed from 3.0 onwards
*/
@Deprecated
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS",
justification = "Writables are going away and this has been this way forever")
public static class KeyValueImporter extends TableMapper<ImmutableBytesWritable, KeyValue> {
private Map<byte[], byte[]> cfRenameMap;
private Filter filter;
private static final Logger LOG = LoggerFactory.getLogger(KeyValueImporter.class);
/**
* @param row The current table row key.
* @param value The columns.
* @param context The current context.
* @throws IOException When something is broken with the data.
*/
@Override
public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException {
try {
if (LOG.isTraceEnabled()) {
LOG.trace(
"Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
}
if (filter == null || !filter.filterRowKey(
PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))) {
for (Cell kv : value.rawCells()) {
kv = filterKv(filter, kv);
// skip if we filtered it out
if (kv == null) continue;
// TODO get rid of ensureKeyValue
context.write(row, KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap)));
}
}
} catch (InterruptedException e) {
e.printStackTrace();
}
}
@Override
public void setup(Context context) {
cfRenameMap = createCfRenameMap(context.getConfiguration());
filter = instantiateFilter(context.getConfiguration());
}
}
public static class CellSortImporter
extends TableMapper<CellWritableComparable, Cell> {
private Map<byte[], byte[]> cfRenameMap;
private Filter filter;
private static final Logger LOG = LoggerFactory.getLogger(CellImporter.class);
/**
* @param row The current table row key.
* @param value The columns.
* @param context The current context.
* @throws IOException When something is broken with the data.
*/
@Override
public void map(ImmutableBytesWritable row, Result value,
Context context)
throws IOException {
try {
if (LOG.isTraceEnabled()) {
LOG.trace("Considering the row."
+ Bytes.toString(row.get(), row.getOffset(), row.getLength()));
}
if (filter == null
|| !filter.filterRowKey(PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(),
(short) row.getLength()))) {
for (Cell kv : value.rawCells()) {
kv = filterKv(filter, kv);
// skip if we filtered it out
if (kv == null) continue;
Cell ret = convertKv(kv, cfRenameMap);
context.write(new CellWritableComparable(ret), ret);
}
}
} catch (InterruptedException e) {
e.printStackTrace();
}
}
@Override
public void setup(Context context) throws IOException {
cfRenameMap = createCfRenameMap(context.getConfiguration());
filter = instantiateFilter(context.getConfiguration());
int reduceNum = context.getNumReduceTasks();
Configuration conf = context.getConfiguration();
TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME));
try (Connection conn = ConnectionFactory.createConnection(conf);
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
byte[][] startKeys = regionLocator.getStartKeys();
if (startKeys.length != reduceNum) {
throw new IOException("Region split after job initialization");
}
CellWritableComparable[] startKeyWraps =
new CellWritableComparable[startKeys.length - 1];
for (int i = 1; i < startKeys.length; ++i) {
startKeyWraps[i - 1] =
new CellWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i]));
}
CellWritableComparablePartitioner.START_KEYS = startKeyWraps;
}
}
}
/**
* A mapper that just writes out KeyValues.
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
justification="Writables are going away and this has been this way forever")
public static class CellImporter extends TableMapper<ImmutableBytesWritable, Cell> {
private Map<byte[], byte[]> cfRenameMap;
private Filter filter;
private static final Logger LOG = LoggerFactory.getLogger(CellImporter.class);
/**
* @param row The current table row key.
* @param value The columns.
* @param context The current context.
* @throws IOException When something is broken with the data.
*/
@Override
public void map(ImmutableBytesWritable row, Result value,
Context context)
throws IOException {
try {
if (LOG.isTraceEnabled()) {
LOG.trace("Considering the row."
+ Bytes.toString(row.get(), row.getOffset(), row.getLength()));
}
if (filter == null
|| !filter.filterRowKey(PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(),
(short) row.getLength()))) {
for (Cell kv : value.rawCells()) {
kv = filterKv(filter, kv);
// skip if we filtered it out
if (kv == null) continue;
context.write(row, new MapReduceExtendedCell(convertKv(kv, cfRenameMap)));
}
}
} catch (InterruptedException e) {
e.printStackTrace();
}
}
@Override
public void setup(Context context) {
cfRenameMap = createCfRenameMap(context.getConfiguration());
filter = instantiateFilter(context.getConfiguration());
}
}
/**
* Write table content out to files in hdfs.
*/
public static class Importer extends TableMapper<ImmutableBytesWritable, Mutation> {
private Map<byte[], byte[]> cfRenameMap;
private List<UUID> clusterIds;
private Filter filter;
private Durability durability;
/**
* @param row The current table row key.
* @param value The columns.
* @param context The current context.
* @throws IOException When something is broken with the data.
*/
@Override
public void map(ImmutableBytesWritable row, Result value,
Context context)
throws IOException {
try {
writeResult(row, value, context);
} catch (InterruptedException e) {
e.printStackTrace();
}
}
private void writeResult(ImmutableBytesWritable key, Result result, Context context)
throws IOException, InterruptedException {
Put put = null;
Delete delete = null;
if (LOG.isTraceEnabled()) {
LOG.trace("Considering the row."
+ Bytes.toString(key.get(), key.getOffset(), key.getLength()));
}
if (filter == null
|| !filter.filterRowKey(PrivateCellUtil.createFirstOnRow(key.get(), key.getOffset(),
(short) key.getLength()))) {
processKV(key, result, context, put, delete);
}
}
protected void processKV(ImmutableBytesWritable key, Result result, Context context, Put put,
Delete delete) throws IOException, InterruptedException {
for (Cell kv : result.rawCells()) {
kv = filterKv(filter, kv);
// skip if we filter it out
if (kv == null) continue;
kv = convertKv(kv, cfRenameMap);
// Deletes and Puts are gathered and written when finished
/*
* If there are sequence of mutations and tombstones in an Export, and after Import the same
* sequence should be restored as it is. If we combine all Delete tombstones into single
* request then there is chance of ignoring few DeleteFamily tombstones, because if we
* submit multiple DeleteFamily tombstones in single Delete request then we are maintaining
* only newest in hbase table and ignoring other. Check - HBASE-12065
*/
if (PrivateCellUtil.isDeleteFamily(kv)) {
Delete deleteFamily = new Delete(key.get());
deleteFamily.add(kv);
if (durability != null) {
deleteFamily.setDurability(durability);
}
deleteFamily.setClusterIds(clusterIds);
context.write(key, deleteFamily);
} else if (CellUtil.isDelete(kv)) {
if (delete == null) {
delete = new Delete(key.get());
}
delete.add(kv);
} else {
if (put == null) {
put = new Put(key.get());
}
addPutToKv(put, kv);
}
}
if (put != null) {
if (durability != null) {
put.setDurability(durability);
}
put.setClusterIds(clusterIds);
context.write(key, put);
}
if (delete != null) {
if (durability != null) {
delete.setDurability(durability);
}
delete.setClusterIds(clusterIds);
context.write(key, delete);
}
}
protected void addPutToKv(Put put, Cell kv) throws IOException {
put.add(kv);
}
@Override
public void setup(Context context) {
LOG.info("Setting up " + getClass() + " mapper.");
Configuration conf = context.getConfiguration();
cfRenameMap = createCfRenameMap(conf);
filter = instantiateFilter(conf);
String durabilityStr = conf.get(WAL_DURABILITY);
if(durabilityStr != null){
durability = Durability.valueOf(durabilityStr.toUpperCase(Locale.ROOT));
LOG.info("setting WAL durability to " + durability);
} else {
LOG.info("setting WAL durability to default.");
}
// TODO: This is kind of ugly doing setup of ZKW just to read the clusterid.
ZKWatcher zkw = null;
Exception ex = null;
try {
zkw = new ZKWatcher(conf, context.getTaskAttemptID().toString(), null);
clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
} catch (ZooKeeperConnectionException e) {
ex = e;
LOG.error("Problem connecting to ZooKeper during task setup", e);
} catch (KeeperException e) {
ex = e;
LOG.error("Problem reading ZooKeeper data during task setup", e);
} catch (IOException e) {
ex = e;
LOG.error("Problem setting up task", e);
} finally {
if (zkw != null) zkw.close();
}
if (clusterIds == null) {
// exit early if setup fails
throw new RuntimeException(ex);
}
}
}
/**
* Create a {@link Filter} to apply to all incoming keys ({@link KeyValue KeyValues}) to
* optionally not include in the job output
* @param conf {@link Configuration} from which to load the filter
* @return the filter to use for the task, or <tt>null</tt> if no filter to should be used
* @throws IllegalArgumentException if the filter is misconfigured
*/
public static Filter instantiateFilter(Configuration conf) {
// get the filter, if it was configured
Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
if (filterClass == null) {
LOG.debug("No configured filter class, accepting all keyvalues.");
return null;
}
LOG.debug("Attempting to create filter:" + filterClass);
String[] filterArgs = conf.getStrings(FILTER_ARGS_CONF_KEY);
ArrayList<byte[]> quotedArgs = toQuotedByteArrays(filterArgs);
try {
Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
return (Filter) m.invoke(null, quotedArgs);
} catch (IllegalAccessException e) {
LOG.error("Couldn't instantiate filter!", e);
throw new RuntimeException(e);
} catch (SecurityException e) {
LOG.error("Couldn't instantiate filter!", e);
throw new RuntimeException(e);
} catch (NoSuchMethodException e) {
LOG.error("Couldn't instantiate filter!", e);
throw new RuntimeException(e);
} catch (IllegalArgumentException e) {
LOG.error("Couldn't instantiate filter!", e);
throw new RuntimeException(e);
} catch (InvocationTargetException e) {
LOG.error("Couldn't instantiate filter!", e);
throw new RuntimeException(e);
}
}
private static ArrayList<byte[]> toQuotedByteArrays(String... stringArgs) {
ArrayList<byte[]> quotedArgs = new ArrayList<>();
for (String stringArg : stringArgs) {
// all the filters' instantiation methods expected quoted args since they are coming from
// the shell, so add them here, though it shouldn't really be needed :-/
quotedArgs.add(Bytes.toBytes("'" + stringArg + "'"));
}
return quotedArgs;
}
/**
* Attempt to filter out the keyvalue
* @param c {@link Cell} on which to apply the filter
* @return <tt>null</tt> if the key should not be written, otherwise returns the original
* {@link Cell}
*/
public static Cell filterKv(Filter filter, Cell c) throws IOException {
// apply the filter and skip this kv if the filter doesn't apply
if (filter != null) {
Filter.ReturnCode code = filter.filterCell(c);
if (LOG.isTraceEnabled()) {
LOG.trace("Filter returned:" + code + " for the cell:" + c);
}
// if its not an accept type, then skip this kv
if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
.equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
return null;
}
}
return c;
}
// helper: create a new KeyValue based on CF rename map
private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {
if(cfRenameMap != null) {
// If there's a rename mapping for this CF, create a new KeyValue
byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
if (newCfName != null) {
kv = new KeyValue(kv.getRowArray(), // row buffer
kv.getRowOffset(), // row offset
kv.getRowLength(), // row length
newCfName, // CF buffer
0, // CF offset
newCfName.length, // CF length
kv.getQualifierArray(), // qualifier buffer
kv.getQualifierOffset(), // qualifier offset
kv.getQualifierLength(), // qualifier length
kv.getTimestamp(), // timestamp
KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type
kv.getValueArray(), // value buffer
kv.getValueOffset(), // value offset
kv.getValueLength()); // value length
}
}
return kv;
}
// helper: make a map from sourceCfName to destCfName by parsing a config key
private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) {
Map<byte[], byte[]> cfRenameMap = null;
String allMappingsPropVal = conf.get(CF_RENAME_PROP);
if(allMappingsPropVal != null) {
// The conf value format should be sourceCf1:destCf1,sourceCf2:destCf2,...
String[] allMappings = allMappingsPropVal.split(",");
for (String mapping: allMappings) {
if(cfRenameMap == null) {
cfRenameMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
}
String [] srcAndDest = mapping.split(":");
if(srcAndDest.length != 2) {
continue;
}
cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
}
}
return cfRenameMap;
}
/**
* <p>Sets a configuration property with key {@link #CF_RENAME_PROP} in conf that tells
* the mapper how to rename column families.
*
* <p>Alternately, instead of calling this function, you could set the configuration key
* {@link #CF_RENAME_PROP} yourself. The value should look like
* <pre>srcCf1:destCf1,srcCf2:destCf2,....</pre>. This would have the same effect on
* the mapper behavior.
*
* @param conf the Configuration in which the {@link #CF_RENAME_PROP} key will be
* set
* @param renameMap a mapping from source CF names to destination CF names
*/
static public void configureCfRenaming(Configuration conf,
Map<String, String> renameMap) {
StringBuilder sb = new StringBuilder();
for(Map.Entry<String,String> entry: renameMap.entrySet()) {
String sourceCf = entry.getKey();
String destCf = entry.getValue();
if(sourceCf.contains(":") || sourceCf.contains(",") ||
destCf.contains(":") || destCf.contains(",")) {
throw new IllegalArgumentException("Illegal character in CF names: "
+ sourceCf + ", " + destCf);
}
if(sb.length() != 0) {
sb.append(",");
}
sb.append(sourceCf + ":" + destCf);
}
conf.set(CF_RENAME_PROP, sb.toString());
}
/**
* Add a Filter to be instantiated on import
* @param conf Configuration to update (will be passed to the job)
* @param clazz {@link Filter} subclass to instantiate on the server.
* @param filterArgs List of arguments to pass to the filter on instantiation
*/
public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz,
List<String> filterArgs) throws IOException {
conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName());
conf.setStrings(Import.FILTER_ARGS_CONF_KEY, filterArgs.toArray(new String[filterArgs.size()]));
}
/**
* Sets up the actual job.
* @param conf The current configuration.
* @param args The command line parameters.
* @return The newly created job.
* @throws IOException When setting up the job fails.
*/
public static Job createSubmittableJob(Configuration conf, String[] args)
throws IOException {
TableName tableName = TableName.valueOf(args[0]);
conf.set(TABLE_NAME, tableName.getNameAsString());
Path inputDir = new Path(args[1]);
Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
job.setJarByClass(Importer.class);
FileInputFormat.setInputPaths(job, inputDir);
job.setInputFormatClass(SequenceFileInputFormat.class);
String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
// make sure we get the filter in the jars
try {
Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
if (filter != null) {
TableMapReduceUtil.addDependencyJarsForClasses(conf, filter);
}
} catch (Exception e) {
throw new IOException(e);
}
if (hfileOutPath != null && conf.getBoolean(HAS_LARGE_RESULT, false)) {
LOG.info("Use Large Result!!");
try (Connection conn = ConnectionFactory.createConnection(conf);
Table table = conn.getTable(tableName);
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
job.setMapperClass(CellSortImporter.class);
job.setReducerClass(CellReducer.class);
Path outputDir = new Path(hfileOutPath);
FileOutputFormat.setOutputPath(job, outputDir);
job.setMapOutputKeyClass(CellWritableComparable.class);
job.setMapOutputValueClass(MapReduceExtendedCell.class);
job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
CellWritableComparable.CellWritableComparator.class,
RawComparator.class);
Path partitionsPath =
new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration()));
FileSystem fs = FileSystem.get(job.getConfiguration());
fs.deleteOnExit(partitionsPath);
job.setPartitionerClass(CellWritableComparablePartitioner.class);
job.setNumReduceTasks(regionLocator.getStartKeys().length);
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class);
}
} else if (hfileOutPath != null) {
LOG.info("writing to hfiles for bulk load.");
job.setMapperClass(CellImporter.class);
try (Connection conn = ConnectionFactory.createConnection(conf);
Table table = conn.getTable(tableName);
RegionLocator regionLocator = conn.getRegionLocator(tableName)){
job.setReducerClass(CellSortReducer.class);
Path outputDir = new Path(hfileOutPath);
FileOutputFormat.setOutputPath(job, outputDir);
job.setMapOutputKeyClass(ImmutableBytesWritable.class);
job.setMapOutputValueClass(MapReduceExtendedCell.class);
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class);
}
} else {
LOG.info("writing directly to table from Mapper.");
// No reducers. Just write straight to table. Call initTableReducerJob
// because it sets up the TableOutputFormat.
job.setMapperClass(Importer.class);
TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job);
job.setNumReduceTasks(0);
}
return job;
}
/*
* @param errorMsg Error message. Can be null.
*/
private static void usage(final String errorMsg) {
if (errorMsg != null && errorMsg.length() > 0) {
System.err.println("ERROR: " + errorMsg);
}
System.err.println("Usage: Import [options] <tablename> <inputdir>");
System.err.println("By default Import will load data directly into HBase. To instead generate");
System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
System.err.println("If there is a large result that includes too much Cell "
+ "whitch can occur OOME caused by the memery sort in reducer, pass the option:");
System.err.println(" -D" + HAS_LARGE_RESULT + "=true");
System.err
.println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
System.err.println(" -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");
System.err.println(" -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");
System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "
+ CF_RENAME_PROP + " property. Futher, filters will only use the"
+ " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "
+ " whether the current row needs to be ignored completely for processing and "
+ " Filter#filterCell(Cell) method to determine if the Cell should be added;"
+ " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including"
+ " the Cell.");
System.err.println("To import data exported from HBase 0.94, use");
System.err.println(" -Dhbase.import.version=0.94");
System.err.println(" -D " + JOB_NAME_CONF_KEY
+ "=jobName - use the specified mapreduce job name for the import");
System.err.println("For performance consider the following options:\n"
+ " -Dmapreduce.map.speculative=false\n"
+ " -Dmapreduce.reduce.speculative=false\n"
+ " -D" + WAL_DURABILITY + "=<Used while writing data to hbase."
+" Allowed values are the supported durability values"
+" like SKIP_WAL/ASYNC_WAL/SYNC_WAL/...>");
}
/**
* If the durability is set to {@link Durability#SKIP_WAL} and the data is imported to hbase, we
* need to flush all the regions of the table as the data is held in memory and is also not
* present in the Write Ahead Log to replay in scenarios of a crash. This method flushes all the
* regions of the table in the scenarios of import data to hbase with {@link Durability#SKIP_WAL}
*/
public static void flushRegionsIfNecessary(Configuration conf) throws IOException,
InterruptedException {
String tableName = conf.get(TABLE_NAME);
Admin hAdmin = null;
Connection connection = null;
String durability = conf.get(WAL_DURABILITY);
// Need to flush if the data is written to hbase and skip wal is enabled.
if (conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null
&& Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) {
LOG.info("Flushing all data that skipped the WAL.");
try {
connection = ConnectionFactory.createConnection(conf);
hAdmin = connection.getAdmin();
hAdmin.flush(TableName.valueOf(tableName));
} finally {
if (hAdmin != null) {
hAdmin.close();
}
if (connection != null) {
connection.close();
}
}
}
}
@Override
public int run(String[] args) throws Exception {
if (args.length < 2) {
usage("Wrong number of arguments: " + args.length);
return -1;
}
String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER);
if (inputVersionString != null) {
getConf().set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString);
}
Job job = createSubmittableJob(getConf(), args);
boolean isJobSuccessful = job.waitForCompletion(true);
if(isJobSuccessful){
// Flush all the regions of the table
flushRegionsIfNecessary(getConf());
}
long inputRecords = job.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue();
long outputRecords = job.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS).getValue();
if (outputRecords < inputRecords) {
System.err.println("Warning, not all records were imported (maybe filtered out).");
if (outputRecords == 0) {
System.err.println("If the data was exported from HBase 0.94 "+
"consider using -Dhbase.import.version=0.94.");
}
}
return (isJobSuccessful ? 0 : 1);
}
/**
* Main entry point.
* @param args The command line parameters.
* @throws Exception When running the job fails.
*/
public static void main(String[] args) throws Exception {
int errCode = ToolRunner.run(HBaseConfiguration.create(), new Import(), args);
System.exit(errCode);
}
}