/
ProtobufUtil.java
1794 lines (1694 loc) · 67.8 KB
/
ProtobufUtil.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
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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.protobuf;
import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC;
import com.google.protobuf.ByteString;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.Message;
import com.google.protobuf.Parser;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.Method;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.function.Function;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.Cell.Type;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.ExtendedCellBuilder;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.SnapshotType;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DynamicClassLoader;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Protobufs utility.
* NOTE: This class OVERLAPS ProtobufUtil in the subpackage 'shaded'. The latter is used
* internally and has more methods. This Class is for Coprocessor Endpoints only though they
* should not be using this private class. It should not be depended upon. Most methods here
* are COPIED from the shaded ProtobufUtils with only difference being they refer to non-shaded
* protobufs.
* @see ProtobufUtil
*/
// TODO: Generate this class from the shaded version.
@InterfaceAudience.Private // TODO: some clients (Hive, etc) use this class.
public final class ProtobufUtil {
private ProtobufUtil() {
}
/**
* Many results are simple: no cell, exists true or false. To save on object creations,
* we reuse them across calls.
*/
// TODO: PICK THESE UP FROM THE SHADED PROTOBUF.
private final static Cell[] EMPTY_CELL_ARRAY = new Cell[]{};
private final static Result EMPTY_RESULT = Result.create(EMPTY_CELL_ARRAY);
final static Result EMPTY_RESULT_EXISTS_TRUE = Result.create(null, true);
final static Result EMPTY_RESULT_EXISTS_FALSE = Result.create(null, false);
private final static Result EMPTY_RESULT_STALE = Result.create(EMPTY_CELL_ARRAY, null, true);
private final static Result EMPTY_RESULT_EXISTS_TRUE_STALE
= Result.create((Cell[])null, true, true);
private final static Result EMPTY_RESULT_EXISTS_FALSE_STALE
= Result.create((Cell[])null, false, true);
private final static ClientProtos.Result EMPTY_RESULT_PB;
private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_TRUE;
private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_FALSE;
private final static ClientProtos.Result EMPTY_RESULT_PB_STALE;
private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_TRUE_STALE;
private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_FALSE_STALE;
static {
ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
builder.setExists(true);
builder.setAssociatedCellCount(0);
EMPTY_RESULT_PB_EXISTS_TRUE = builder.build();
builder.setStale(true);
EMPTY_RESULT_PB_EXISTS_TRUE_STALE = builder.build();
builder.clear();
builder.setExists(false);
builder.setAssociatedCellCount(0);
EMPTY_RESULT_PB_EXISTS_FALSE = builder.build();
builder.setStale(true);
EMPTY_RESULT_PB_EXISTS_FALSE_STALE = builder.build();
builder.clear();
builder.setAssociatedCellCount(0);
EMPTY_RESULT_PB = builder.build();
builder.setStale(true);
EMPTY_RESULT_PB_STALE = builder.build();
}
/**
* Dynamic class loader to load filter/comparators
*/
private final static class ClassLoaderHolder {
private final static ClassLoader CLASS_LOADER;
static {
ClassLoader parent = ProtobufUtil.class.getClassLoader();
Configuration conf = HBaseConfiguration.create();
CLASS_LOADER = AccessController.doPrivileged((PrivilegedAction<ClassLoader>)
() -> new DynamicClassLoader(conf, parent)
);
}
}
/**
* Prepend the passed bytes with four bytes of magic, {@link ProtobufMagic#PB_MAGIC},
* to flag what follows as a protobuf in hbase. Prepend these bytes to all content written to
* znodes, etc.
* @param bytes Bytes to decorate
* @return The passed <code>bytes</code> with magic prepended (Creates a new
* byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length.
*/
public static byte [] prependPBMagic(final byte [] bytes) {
return Bytes.add(PB_MAGIC, bytes);
}
/**
* @param bytes Bytes to check.
* @return True if passed <code>bytes</code> has {@link ProtobufMagic#PB_MAGIC} for a prefix.
*/
public static boolean isPBMagicPrefix(final byte [] bytes) {
return ProtobufMagic.isPBMagicPrefix(bytes);
}
/**
* @param bytes Bytes to check.
* @param offset offset to start at
* @param len length to use
* @return True if passed <code>bytes</code> has {@link ProtobufMagic#PB_MAGIC} for a prefix.
*/
public static boolean isPBMagicPrefix(final byte [] bytes, int offset, int len) {
return ProtobufMagic.isPBMagicPrefix(bytes, offset, len);
}
/**
* @param bytes bytes to check
* @throws DeserializationException if we are missing the pb magic prefix
*/
public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException {
if (!isPBMagicPrefix(bytes)) {
String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length);
throw new DeserializationException(
"Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix, bytes: " + bytesPrefix);
}
}
/**
* @return Length of {@link ProtobufMagic#lengthOfPBMagic()}
*/
public static int lengthOfPBMagic() {
return ProtobufMagic.lengthOfPBMagic();
}
/**
* Return the IOException thrown by the remote server wrapped in
* ServiceException as cause.
*
* @param se ServiceException that wraps IO exception thrown by the server
* @return Exception wrapped in ServiceException or
* a new IOException that wraps the unexpected ServiceException.
*/
public static IOException getRemoteException(ServiceException se) {
return makeIOExceptionOfException(se);
}
/**
* Return the Exception thrown by the remote server wrapped in
* ServiceException as cause. RemoteException are left untouched.
*
* @param e ServiceException that wraps IO exception thrown by the server
* @return Exception wrapped in ServiceException.
*/
public static IOException getServiceException(org.apache.hbase.thirdparty.com.google.protobuf.ServiceException e) {
Throwable t = e.getCause();
if (ExceptionUtil.isInterrupt(t)) {
return ExceptionUtil.asInterrupt(t);
}
return t instanceof IOException ? (IOException) t : new HBaseIOException(t);
}
/**
* Like {@link #getRemoteException(ServiceException)} but more generic, able to handle more than
* just {@link ServiceException}. Prefer this method to
* {@link #getRemoteException(ServiceException)} because trying to
* contain direct protobuf references.
*/
public static IOException handleRemoteException(Throwable e) {
return makeIOExceptionOfException(e);
}
private static IOException makeIOExceptionOfException(Throwable e) {
Throwable t = e;
if (e instanceof ServiceException ||
e instanceof org.apache.hbase.thirdparty.com.google.protobuf.ServiceException) {
t = e.getCause();
}
if (ExceptionUtil.isInterrupt(t)) {
return ExceptionUtil.asInterrupt(t);
}
if (t instanceof RemoteException) {
t = ((RemoteException)t).unwrapRemoteException();
}
return t instanceof IOException? (IOException)t: new HBaseIOException(t);
}
/**
* Convert a ServerName to a protocol buffer ServerName
*
* @param serverName the ServerName to convert
* @return the converted protocol buffer ServerName
* @see #toServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName)
*/
public static HBaseProtos.ServerName
toServerName(final ServerName serverName) {
if (serverName == null) return null;
HBaseProtos.ServerName.Builder builder =
HBaseProtos.ServerName.newBuilder();
builder.setHostName(serverName.getHostname());
if (serverName.getPort() >= 0) {
builder.setPort(serverName.getPort());
}
if (serverName.getStartcode() >= 0) {
builder.setStartCode(serverName.getStartcode());
}
return builder.build();
}
/**
* Convert a protocol buffer ServerName to a ServerName
*
* @param proto the protocol buffer ServerName to convert
* @return the converted ServerName
*/
public static ServerName toServerName(final HBaseProtos.ServerName proto) {
if (proto == null) return null;
String hostName = proto.getHostName();
long startCode = -1;
int port = -1;
if (proto.hasPort()) {
port = proto.getPort();
}
if (proto.hasStartCode()) {
startCode = proto.getStartCode();
}
return ServerName.valueOf(hostName, port, startCode);
}
/**
* Convert a protobuf Durability into a client Durability
*/
public static Durability toDurability(
final ClientProtos.MutationProto.Durability proto) {
switch(proto) {
case USE_DEFAULT:
return Durability.USE_DEFAULT;
case SKIP_WAL:
return Durability.SKIP_WAL;
case ASYNC_WAL:
return Durability.ASYNC_WAL;
case SYNC_WAL:
return Durability.SYNC_WAL;
case FSYNC_WAL:
return Durability.FSYNC_WAL;
default:
return Durability.USE_DEFAULT;
}
}
/**
* Convert a client Durability into a protbuf Durability
*/
public static ClientProtos.MutationProto.Durability toDurability(
final Durability d) {
switch(d) {
case USE_DEFAULT:
return ClientProtos.MutationProto.Durability.USE_DEFAULT;
case SKIP_WAL:
return ClientProtos.MutationProto.Durability.SKIP_WAL;
case ASYNC_WAL:
return ClientProtos.MutationProto.Durability.ASYNC_WAL;
case SYNC_WAL:
return ClientProtos.MutationProto.Durability.SYNC_WAL;
case FSYNC_WAL:
return ClientProtos.MutationProto.Durability.FSYNC_WAL;
default:
return ClientProtos.MutationProto.Durability.USE_DEFAULT;
}
}
/**
* Convert a protocol buffer Get to a client Get
*
* @param proto the protocol buffer Get to convert
* @return the converted client Get
* @throws IOException
*/
public static Get toGet(final ClientProtos.Get proto) throws IOException {
if (proto == null) return null;
byte[] row = proto.getRow().toByteArray();
Get get = new Get(row);
if (proto.hasCacheBlocks()) {
get.setCacheBlocks(proto.getCacheBlocks());
}
if (proto.hasMaxVersions()) {
get.readVersions(proto.getMaxVersions());
}
if (proto.hasStoreLimit()) {
get.setMaxResultsPerColumnFamily(proto.getStoreLimit());
}
if (proto.hasStoreOffset()) {
get.setRowOffsetPerColumnFamily(proto.getStoreOffset());
}
if (proto.getCfTimeRangeCount() > 0) {
for (HBaseProtos.ColumnFamilyTimeRange cftr : proto.getCfTimeRangeList()) {
TimeRange timeRange = protoToTimeRange(cftr.getTimeRange());
get.setColumnFamilyTimeRange(cftr.getColumnFamily().toByteArray(),
timeRange.getMin(), timeRange.getMax());
}
}
if (proto.hasTimeRange()) {
TimeRange timeRange = protoToTimeRange(proto.getTimeRange());
get.setTimeRange(timeRange.getMin(), timeRange.getMax());
}
if (proto.hasFilter()) {
FilterProtos.Filter filter = proto.getFilter();
get.setFilter(ProtobufUtil.toFilter(filter));
}
for (NameBytesPair attribute: proto.getAttributeList()) {
get.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
if (proto.getColumnCount() > 0) {
for (Column column: proto.getColumnList()) {
byte[] family = column.getFamily().toByteArray();
if (column.getQualifierCount() > 0) {
for (ByteString qualifier: column.getQualifierList()) {
get.addColumn(family, qualifier.toByteArray());
}
} else {
get.addFamily(family);
}
}
}
if (proto.hasExistenceOnly() && proto.getExistenceOnly()){
get.setCheckExistenceOnly(true);
}
if (proto.hasConsistency()) {
get.setConsistency(toConsistency(proto.getConsistency()));
}
if (proto.hasLoadColumnFamiliesOnDemand()) {
get.setLoadColumnFamiliesOnDemand(proto.getLoadColumnFamiliesOnDemand());
}
return get;
}
public static Consistency toConsistency(ClientProtos.Consistency consistency) {
switch (consistency) {
case STRONG : return Consistency.STRONG;
case TIMELINE : return Consistency.TIMELINE;
default : return Consistency.STRONG;
}
}
public static ClientProtos.Consistency toConsistency(Consistency consistency) {
switch (consistency) {
case STRONG : return ClientProtos.Consistency.STRONG;
case TIMELINE : return ClientProtos.Consistency.TIMELINE;
default : return ClientProtos.Consistency.STRONG;
}
}
/**
* Convert a protocol buffer Mutate to a Put.
*
* @param proto The protocol buffer MutationProto to convert
* @return A client Put.
* @throws IOException
*/
public static Put toPut(final MutationProto proto)
throws IOException {
return toPut(proto, null);
}
/**
* Convert a protocol buffer Mutate to a Put.
*
* @param proto The protocol buffer MutationProto to convert
* @param cellScanner If non-null, the Cell data that goes with this proto.
* @return A client Put.
* @throws IOException
*/
public static Put toPut(final MutationProto proto, final CellScanner cellScanner)
throws IOException {
// TODO: Server-side at least why do we convert back to the Client types? Why not just pb it?
MutationType type = proto.getMutateType();
assert type == MutationType.PUT: type.name();
long timestamp = proto.hasTimestamp()? proto.getTimestamp(): HConstants.LATEST_TIMESTAMP;
Put put = proto.hasRow() ? new Put(proto.getRow().toByteArray(), timestamp) : null;
int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
if (cellCount > 0) {
// The proto has metadata only and the data is separate to be found in the cellScanner.
if (cellScanner == null) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
toShortString(proto));
}
for (int i = 0; i < cellCount; i++) {
if (!cellScanner.advance()) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
" no cell returned: " + toShortString(proto));
}
Cell cell = cellScanner.current();
if (put == null) {
put = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
}
put.add(cell);
}
} else {
if (put == null) {
throw new IllegalArgumentException("row cannot be null");
}
// The proto has the metadata and the data itself
ExtendedCellBuilder cellBuilder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) {
if (!qv.hasValue()) {
throw new DoNotRetryIOException(
"Missing required field: qualifier value");
}
long ts = timestamp;
if (qv.hasTimestamp()) {
ts = qv.getTimestamp();
}
byte[] allTagsBytes;
if (qv.hasTags()) {
allTagsBytes = qv.getTags().toByteArray();
if(qv.hasDeleteType()) {
put.add(cellBuilder.clear()
.setRow(put.getRow())
.setFamily(family)
.setQualifier(qv.hasQualifier() ? qv.getQualifier().toByteArray() : null)
.setTimestamp(ts)
.setType(fromDeleteType(qv.getDeleteType()).getCode())
.setTags(allTagsBytes)
.build());
} else {
put.add(cellBuilder.clear()
.setRow(put.getRow())
.setFamily(family)
.setQualifier(qv.hasQualifier() ? qv.getQualifier().toByteArray() : null)
.setTimestamp(ts)
.setType(Cell.Type.Put)
.setValue(qv.hasValue() ? qv.getValue().toByteArray() : null)
.setTags(allTagsBytes)
.build());
}
} else {
if(qv.hasDeleteType()) {
put.add(cellBuilder.clear()
.setRow(put.getRow())
.setFamily(family)
.setQualifier(qv.hasQualifier() ? qv.getQualifier().toByteArray() : null)
.setTimestamp(ts)
.setType(fromDeleteType(qv.getDeleteType()).getCode())
.build());
} else{
put.add(cellBuilder.clear()
.setRow(put.getRow())
.setFamily(family)
.setQualifier(qv.hasQualifier() ? qv.getQualifier().toByteArray() : null)
.setTimestamp(ts)
.setType(Type.Put)
.setValue(qv.hasValue() ? qv.getValue().toByteArray() : null)
.build());
}
}
}
}
}
put.setDurability(toDurability(proto.getDurability()));
for (NameBytesPair attribute: proto.getAttributeList()) {
put.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
return put;
}
/**
* Convert a protocol buffer Mutate to a Delete
*
* @param proto the protocol buffer Mutate to convert
* @return the converted client Delete
* @throws IOException
*/
public static Delete toDelete(final MutationProto proto)
throws IOException {
return toDelete(proto, null);
}
/**
* Convert a protocol buffer Mutate to a Delete
*
* @param proto the protocol buffer Mutate to convert
* @param cellScanner if non-null, the data that goes with this delete.
* @return the converted client Delete
* @throws IOException
*/
public static Delete toDelete(final MutationProto proto, final CellScanner cellScanner)
throws IOException {
MutationType type = proto.getMutateType();
assert type == MutationType.DELETE : type.name();
long timestamp = proto.hasTimestamp() ? proto.getTimestamp() : HConstants.LATEST_TIMESTAMP;
Delete delete = proto.hasRow() ? new Delete(proto.getRow().toByteArray(), timestamp) : null;
int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
if (cellCount > 0) {
// The proto has metadata only and the data is separate to be found in the cellScanner.
if (cellScanner == null) {
// TextFormat should be fine for a Delete since it carries no data, just coordinates.
throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
TextFormat.shortDebugString(proto));
}
for (int i = 0; i < cellCount; i++) {
if (!cellScanner.advance()) {
// TextFormat should be fine for a Delete since it carries no data, just coordinates.
throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
" no cell returned: " + TextFormat.shortDebugString(proto));
}
Cell cell = cellScanner.current();
if (delete == null) {
delete =
new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
}
delete.add(cell);
}
} else {
if (delete == null) {
throw new IllegalArgumentException("row cannot be null");
}
for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) {
DeleteType deleteType = qv.getDeleteType();
byte[] qualifier = null;
if (qv.hasQualifier()) {
qualifier = qv.getQualifier().toByteArray();
}
long ts = HConstants.LATEST_TIMESTAMP;
if (qv.hasTimestamp()) {
ts = qv.getTimestamp();
}
if (deleteType == DeleteType.DELETE_ONE_VERSION) {
delete.addColumn(family, qualifier, ts);
} else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) {
delete.addColumns(family, qualifier, ts);
} else if (deleteType == DeleteType.DELETE_FAMILY_VERSION) {
delete.addFamilyVersion(family, ts);
} else {
delete.addFamily(family, ts);
}
}
}
}
delete.setDurability(toDurability(proto.getDurability()));
for (NameBytesPair attribute: proto.getAttributeList()) {
delete.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
return delete;
}
@FunctionalInterface
private interface ConsumerWithException <T, U> {
void accept(T t, U u) throws IOException;
}
private static <T extends Mutation> T toDelta(Function<Bytes, T> supplier, ConsumerWithException<T, Cell> consumer,
final MutationProto proto, final CellScanner cellScanner) throws IOException {
byte[] row = proto.hasRow() ? proto.getRow().toByteArray() : null;
T mutation = row == null ? null : supplier.apply(new Bytes(row));
int cellCount = proto.hasAssociatedCellCount() ? proto.getAssociatedCellCount() : 0;
if (cellCount > 0) {
// The proto has metadata only and the data is separate to be found in the cellScanner.
if (cellScanner == null) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
toShortString(proto));
}
for (int i = 0; i < cellCount; i++) {
if (!cellScanner.advance()) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
" no cell returned: " + toShortString(proto));
}
Cell cell = cellScanner.current();
if (mutation == null) {
mutation = supplier.apply(new Bytes(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
}
consumer.accept(mutation, cell);
}
} else {
if (mutation == null) {
throw new IllegalArgumentException("row cannot be null");
}
for (ColumnValue column : proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv : column.getQualifierValueList()) {
byte[] qualifier = qv.getQualifier().toByteArray();
if (!qv.hasValue()) {
throw new DoNotRetryIOException(
"Missing required field: qualifier value");
}
byte[] value = qv.getValue().toByteArray();
byte[] tags = null;
if (qv.hasTags()) {
tags = qv.getTags().toByteArray();
}
consumer.accept(mutation, ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setRow(mutation.getRow()).setFamily(family)
.setQualifier(qualifier).setTimestamp(qv.getTimestamp())
.setType(KeyValue.Type.Put.getCode()).setValue(value)
.setTags(tags).setSequenceId(0)
.build());
}
}
}
mutation.setDurability(toDurability(proto.getDurability()));
for (NameBytesPair attribute : proto.getAttributeList()) {
mutation.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
return mutation;
}
/**
* Convert a protocol buffer Mutate to an Append
* @param cellScanner
* @param proto the protocol buffer Mutate to convert
* @return the converted client Append
* @throws IOException
*/
public static Append toAppend(final MutationProto proto, final CellScanner cellScanner)
throws IOException {
MutationType type = proto.getMutateType();
assert type == MutationType.APPEND : type.name();
Append append = toDelta((Bytes row) -> new Append(row.get(), row.getOffset(), row.getLength()),
Append::add, proto, cellScanner);
if (proto.hasTimeRange()) {
TimeRange timeRange = protoToTimeRange(proto.getTimeRange());
append.setTimeRange(timeRange.getMin(), timeRange.getMax());
}
return append;
}
/**
* Convert a protocol buffer Mutate to an Increment
*
* @param proto the protocol buffer Mutate to convert
* @return the converted client Increment
* @throws IOException
*/
public static Increment toIncrement(final MutationProto proto, final CellScanner cellScanner)
throws IOException {
MutationType type = proto.getMutateType();
assert type == MutationType.INCREMENT : type.name();
Increment increment = toDelta((Bytes row) -> new Increment(row.get(), row.getOffset(), row.getLength()),
Increment::add, proto, cellScanner);
if (proto.hasTimeRange()) {
TimeRange timeRange = protoToTimeRange(proto.getTimeRange());
increment.setTimeRange(timeRange.getMin(), timeRange.getMax());
}
return increment;
}
/**
* Convert a MutateRequest to Mutation
*
* @param proto the protocol buffer Mutate to convert
* @return the converted Mutation
* @throws IOException
*/
public static Mutation toMutation(final MutationProto proto) throws IOException {
MutationType type = proto.getMutateType();
if (type == MutationType.APPEND) {
return toAppend(proto, null);
}
if (type == MutationType.DELETE) {
return toDelete(proto, null);
}
if (type == MutationType.PUT) {
return toPut(proto, null);
}
throw new IOException("Unknown mutation type " + type);
}
/**
* Convert a protocol buffer Mutate to a Get.
* @param proto the protocol buffer Mutate to convert.
* @param cellScanner
* @return the converted client get.
* @throws IOException
*/
public static Get toGet(final MutationProto proto, final CellScanner cellScanner)
throws IOException {
MutationType type = proto.getMutateType();
assert type == MutationType.INCREMENT || type == MutationType.APPEND : type.name();
byte[] row = proto.hasRow() ? proto.getRow().toByteArray() : null;
Get get = null;
int cellCount = proto.hasAssociatedCellCount() ? proto.getAssociatedCellCount() : 0;
if (cellCount > 0) {
// The proto has metadata only and the data is separate to be found in the cellScanner.
if (cellScanner == null) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: "
+ TextFormat.shortDebugString(proto));
}
for (int i = 0; i < cellCount; i++) {
if (!cellScanner.advance()) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i
+ " no cell returned: " + TextFormat.shortDebugString(proto));
}
Cell cell = cellScanner.current();
if (get == null) {
get = new Get(Bytes.copy(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
}
get.addColumn(
Bytes.copy(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()),
Bytes.copy(cell.getQualifierArray(), cell.getQualifierOffset(),
cell.getQualifierLength()));
}
} else {
get = new Get(row);
for (ColumnValue column : proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv : column.getQualifierValueList()) {
byte[] qualifier = qv.getQualifier().toByteArray();
if (!qv.hasValue()) {
throw new DoNotRetryIOException("Missing required field: qualifier value");
}
get.addColumn(family, qualifier);
}
}
}
if (proto.hasTimeRange()) {
TimeRange timeRange = protoToTimeRange(proto.getTimeRange());
get.setTimeRange(timeRange.getMin(), timeRange.getMax());
}
for (NameBytesPair attribute : proto.getAttributeList()) {
get.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
return get;
}
public static ClientProtos.Scan.ReadType toReadType(Scan.ReadType readType) {
switch (readType) {
case DEFAULT:
return ClientProtos.Scan.ReadType.DEFAULT;
case STREAM:
return ClientProtos.Scan.ReadType.STREAM;
case PREAD:
return ClientProtos.Scan.ReadType.PREAD;
default:
throw new IllegalArgumentException("Unknown ReadType: " + readType);
}
}
public static Scan.ReadType toReadType(ClientProtos.Scan.ReadType readType) {
switch (readType) {
case DEFAULT:
return Scan.ReadType.DEFAULT;
case STREAM:
return Scan.ReadType.STREAM;
case PREAD:
return Scan.ReadType.PREAD;
default:
throw new IllegalArgumentException("Unknown ReadType: " + readType);
}
}
/**
* Convert a client Scan to a protocol buffer Scan
*
* @param scan the client Scan to convert
* @return the converted protocol buffer Scan
* @throws IOException
*/
public static ClientProtos.Scan toScan(
final Scan scan) throws IOException {
ClientProtos.Scan.Builder scanBuilder =
ClientProtos.Scan.newBuilder();
scanBuilder.setCacheBlocks(scan.getCacheBlocks());
if (scan.getBatch() > 0) {
scanBuilder.setBatchSize(scan.getBatch());
}
if (scan.getMaxResultSize() > 0) {
scanBuilder.setMaxResultSize(scan.getMaxResultSize());
}
if (scan.isSmall()) {
scanBuilder.setSmall(scan.isSmall());
}
if (scan.getAllowPartialResults()) {
scanBuilder.setAllowPartialResults(scan.getAllowPartialResults());
}
Boolean loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue();
if (loadColumnFamiliesOnDemand != null) {
scanBuilder.setLoadColumnFamiliesOnDemand(loadColumnFamiliesOnDemand);
}
scanBuilder.setMaxVersions(scan.getMaxVersions());
scan.getColumnFamilyTimeRange().forEach((cf, timeRange) -> {
scanBuilder.addCfTimeRange(HBaseProtos.ColumnFamilyTimeRange.newBuilder()
.setColumnFamily(ByteStringer.wrap(cf))
.setTimeRange(toTimeRange(timeRange))
.build());
});
scanBuilder.setTimeRange(toTimeRange(scan.getTimeRange()));
Map<String, byte[]> attributes = scan.getAttributesMap();
if (!attributes.isEmpty()) {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
scanBuilder.addAttribute(attributeBuilder.build());
}
}
byte[] startRow = scan.getStartRow();
if (startRow != null && startRow.length > 0) {
scanBuilder.setStartRow(ByteStringer.wrap(startRow));
}
byte[] stopRow = scan.getStopRow();
if (stopRow != null && stopRow.length > 0) {
scanBuilder.setStopRow(ByteStringer.wrap(stopRow));
}
if (scan.hasFilter()) {
scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
}
if (scan.hasFamilies()) {
Column.Builder columnBuilder = Column.newBuilder();
for (Map.Entry<byte[],NavigableSet<byte []>>
family: scan.getFamilyMap().entrySet()) {
columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
NavigableSet<byte []> qualifiers = family.getValue();
columnBuilder.clearQualifier();
if (qualifiers != null && qualifiers.size() > 0) {
for (byte [] qualifier: qualifiers) {
columnBuilder.addQualifier(ByteStringer.wrap(qualifier));
}
}
scanBuilder.addColumn(columnBuilder.build());
}
}
if (scan.getMaxResultsPerColumnFamily() >= 0) {
scanBuilder.setStoreLimit(scan.getMaxResultsPerColumnFamily());
}
if (scan.getRowOffsetPerColumnFamily() > 0) {
scanBuilder.setStoreOffset(scan.getRowOffsetPerColumnFamily());
}
if (scan.isReversed()) {
scanBuilder.setReversed(scan.isReversed());
}
if (scan.getConsistency() == Consistency.TIMELINE) {
scanBuilder.setConsistency(toConsistency(scan.getConsistency()));
}
if (scan.getCaching() > 0) {
scanBuilder.setCaching(scan.getCaching());
}
long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);
if (mvccReadPoint > 0) {
scanBuilder.setMvccReadPoint(mvccReadPoint);
}
if (!scan.includeStartRow()) {
scanBuilder.setIncludeStartRow(false);
}
scanBuilder.setIncludeStopRow(scan.includeStopRow());
if (scan.getReadType() != Scan.ReadType.DEFAULT) {
scanBuilder.setReadType(toReadType(scan.getReadType()));
}
return scanBuilder.build();
}
/**
* Convert a protocol buffer Scan to a client Scan
*
* @param proto the protocol buffer Scan to convert
* @return the converted client Scan
* @throws IOException
*/
public static Scan toScan(
final ClientProtos.Scan proto) throws IOException {
byte[] startRow = HConstants.EMPTY_START_ROW;
byte[] stopRow = HConstants.EMPTY_END_ROW;
boolean includeStartRow = true;
boolean includeStopRow = false;
if (proto.hasStartRow()) {
startRow = proto.getStartRow().toByteArray();
}
if (proto.hasStopRow()) {
stopRow = proto.getStopRow().toByteArray();
}
if (proto.hasIncludeStartRow()) {
includeStartRow = proto.getIncludeStartRow();
}
if (proto.hasIncludeStopRow()) {
includeStopRow = proto.getIncludeStopRow();
}
Scan scan =
new Scan().withStartRow(startRow, includeStartRow).withStopRow(stopRow, includeStopRow);
if (proto.hasCacheBlocks()) {
scan.setCacheBlocks(proto.getCacheBlocks());
}
if (proto.hasMaxVersions()) {
scan.setMaxVersions(proto.getMaxVersions());
}
if (proto.hasStoreLimit()) {
scan.setMaxResultsPerColumnFamily(proto.getStoreLimit());
}
if (proto.hasStoreOffset()) {
scan.setRowOffsetPerColumnFamily(proto.getStoreOffset());
}
if (proto.hasLoadColumnFamiliesOnDemand()) {
scan.setLoadColumnFamiliesOnDemand(proto.getLoadColumnFamiliesOnDemand());
}
if (proto.getCfTimeRangeCount() > 0) {
for (HBaseProtos.ColumnFamilyTimeRange cftr : proto.getCfTimeRangeList()) {
TimeRange timeRange = protoToTimeRange(cftr.getTimeRange());
scan.setColumnFamilyTimeRange(cftr.getColumnFamily().toByteArray(),
timeRange.getMin(), timeRange.getMax());
}
}
if (proto.hasTimeRange()) {
TimeRange timeRange = protoToTimeRange(proto.getTimeRange());