23
23
import com .alipay .oceanbase .hbase .filter .HBaseFilterUtils ;
24
24
import com .alipay .oceanbase .hbase .result .ClientStreamScanner ;
25
25
import com .alipay .oceanbase .hbase .util .*;
26
+ import com .alipay .oceanbase .rpc .ObGlobal ;
26
27
import com .alipay .oceanbase .rpc .ObTableClient ;
27
28
import com .alipay .oceanbase .rpc .exception .ObTableException ;
28
29
import com .alipay .oceanbase .rpc .location .model .partition .Partition ;
30
+ import com .alipay .oceanbase .rpc .exception .ObTableUnexpectedException ;
29
31
import com .alipay .oceanbase .rpc .mutation .BatchOperation ;
30
32
import com .alipay .oceanbase .rpc .mutation .result .BatchOperationResult ;
33
+ import com .alipay .oceanbase .rpc .mutation .result .MutationResult ;
34
+ import com .alipay .oceanbase .rpc .protocol .payload .ObPayload ;
31
35
import com .alipay .oceanbase .rpc .protocol .payload .impl .ObObj ;
32
36
import com .alipay .oceanbase .rpc .protocol .payload .impl .ObRowKey ;
33
37
import com .alipay .oceanbase .rpc .protocol .payload .impl .execute .*;
40
44
import com .alipay .oceanbase .rpc .stream .ObTableClientQueryStreamResult ;
41
45
import com .alipay .oceanbase .rpc .table .ObHBaseParams ;
42
46
import com .alipay .oceanbase .rpc .table .ObKVParams ;
47
+ import com .alipay .oceanbase .rpc .table .ObTableClientQueryImpl ;
43
48
44
49
import com .google .protobuf .Descriptors ;
45
50
import com .google .protobuf .Message ;
46
51
import com .google .protobuf .Service ;
47
52
import com .google .protobuf .ServiceException ;
53
+ import jdk .nashorn .internal .objects .Global ;
48
54
import org .apache .hadoop .classification .InterfaceAudience ;
49
55
import org .apache .hadoop .conf .Configuration ;
50
56
import org .apache .hadoop .hbase .*;
67
73
import static com .alipay .oceanbase .hbase .constants .OHConstants .*;
68
74
import static com .alipay .oceanbase .hbase .util .Preconditions .checkArgument ;
69
75
import static com .alipay .oceanbase .hbase .util .TableHBaseLoggerFactory .LCD ;
76
+ import static com .alipay .oceanbase .rpc .mutation .MutationFactory .colVal ;
77
+ import static com .alipay .oceanbase .rpc .mutation .MutationFactory .row ;
70
78
import static com .alipay .oceanbase .rpc .protocol .payload .impl .execute .ObTableOperation .getInstance ;
71
79
import static com .alipay .oceanbase .rpc .protocol .payload .impl .execute .ObTableOperationType .*;
72
80
import static com .alipay .sofa .common .thread .SofaThreadPoolConstants .SOFA_THREAD_POOL_LOGGING_CAPABILITY ;
@@ -458,16 +466,19 @@ public boolean exists(Get get) throws IOException {
458
466
459
467
@ Override
460
468
public boolean [] existsAll (List <Get > gets ) throws IOException {
461
- if (gets .isEmpty ()) {
462
- return new boolean [] {};
463
- }
464
- if (gets .size () == 1 ) {
465
- return new boolean [] { exists (gets .get (0 )) };
466
- }
467
- Result [] r = get (gets );
468
- boolean [] ret = new boolean [r .length ];
469
- for (int i = 0 ; i < gets .size (); ++i ) {
470
- ret [i ] = exists (gets .get (i ));
469
+ boolean [] ret = new boolean [gets .size ()];
470
+ List <Get > newGets = new ArrayList <>();
471
+ // if just checkExistOnly, batch get will not return any result or row count
472
+ // therefore we have to set checkExistOnly as false and so the result can be returned
473
+ // TODO: adjust ExistOnly in server when using batch get
474
+ for (Get get : gets ) {
475
+ Get newGet = new Get (get );
476
+ newGet .setCheckExistenceOnly (false );
477
+ newGets .add (newGet );
478
+ }
479
+ Result [] results = get (newGets );
480
+ for (int i = 0 ; i < results .length ; ++i ) {
481
+ ret [i ] = !results [i ].isEmpty ();
471
482
}
472
483
return ret ;
473
484
}
@@ -625,7 +636,7 @@ private void compatOldServerBatch(final List<? extends Row> actions, final Objec
625
636
}
626
637
627
638
@ Override
628
- public void batch (final List <? extends Row > actions , final Object [] results ) throws IOException {
639
+ public void batch (final List <? extends Row > actions , final Object [] results ) throws IOException {
629
640
if (actions == null ) {
630
641
return ;
631
642
}
@@ -637,7 +648,7 @@ public void batch(final List<? extends Row> actions, final Object[] results) thr
637
648
BatchError batchError = new BatchError ();
638
649
obTableClient .setRuntimeBatchExecutor (executePool );
639
650
List <Integer > resultMapSingleOp = new LinkedList <>();
640
- if (!CompatibilityUtil . isBatchSupport ()) {
651
+ if (!ObGlobal . isHBaseBatchSupport ()) {
641
652
try {
642
653
compatOldServerBatch (actions , results , batchError );
643
654
} catch (Exception e ) {
@@ -660,6 +671,23 @@ public void batch(final List<? extends Row> actions, final Object[] results) thr
660
671
results [i ] = tmpResults .getResults ().get (index );
661
672
}
662
673
batchError .add ((ObTableException ) tmpResults .getResults ().get (index ), actions .get (i ), null );
674
+ } else if (actions .get (i ) instanceof Get ) {
675
+ if (results != null ) {
676
+ // get results have been wrapped in MutationResult, need to fetch it
677
+ if (tmpResults .getResults ().get (index ) instanceof MutationResult ) {
678
+ MutationResult mutationResult = (MutationResult ) tmpResults .getResults ().get (index );
679
+ ObPayload innerResult = mutationResult .getResult ();
680
+ if (innerResult instanceof ObTableSingleOpResult ) {
681
+ ObTableSingleOpResult singleOpResult = (ObTableSingleOpResult ) innerResult ;
682
+ List <Cell > cells = generateGetResult (singleOpResult );
683
+ results [i ] = Result .create (cells );
684
+ } else {
685
+ throw new ObTableUnexpectedException ("Unexpected type of result in MutationResult" );
686
+ }
687
+ } else {
688
+ throw new ObTableUnexpectedException ("Unexpected type of result in batch" );
689
+ }
690
+ }
663
691
} else {
664
692
if (results != null ) {
665
693
results [i ] = new Result ();
@@ -673,17 +701,58 @@ public void batch(final List<? extends Row> actions, final Object[] results) thr
673
701
}
674
702
}
675
703
704
+ private List <Cell > generateGetResult (ObTableSingleOpResult getResult ) throws IOException {
705
+ List <Cell > cells = new ArrayList <>();
706
+ ObTableSingleOpEntity singleOpEntity = getResult .getEntity ();
707
+ // all values queried by this get are contained in properties
708
+ // qualifier in batch get result is always appended after family
709
+ List <ObObj > propertiesValues = singleOpEntity .getPropertiesValues ();
710
+ int valueIdx = 0 ;
711
+ while (valueIdx < propertiesValues .size ()) {
712
+ // values in propertiesValues like: [ K, Q, T, V, K, Q, T, V ... ]
713
+ // we need to retrieve K Q T V and construct them to cells: [ cell_0, cell_1, ... ]
714
+ byte [][] familyAndQualifier = new byte [2 ][];
715
+ try {
716
+ // split family and qualifier
717
+ familyAndQualifier = OHBaseFuncUtils
718
+ .extractFamilyFromQualifier ((byte []) propertiesValues .get (valueIdx + 1 ).getValue ());
719
+ } catch (Exception e ) {
720
+ throw new IOException (e );
721
+ }
722
+ KeyValue kv = new KeyValue ((byte []) propertiesValues .get (valueIdx ).getValue (),//K
723
+ familyAndQualifier [0 ], // family
724
+ familyAndQualifier [1 ], // qualifiermat
725
+ (Long ) propertiesValues .get (valueIdx + 2 ).getValue (), // T
726
+ (byte []) propertiesValues .get (valueIdx + 3 ).getValue ()// V
727
+ );
728
+ cells .add (kv );
729
+ valueIdx += 4 ;
730
+ }
731
+ return cells ;
732
+ }
733
+
734
+
676
735
private String getTargetTableName (List <? extends Row > actions ) {
677
736
byte [] family = null ;
678
737
for (Row action : actions ) {
679
738
if (action instanceof RowMutations || action instanceof RegionCoprocessorServiceExec ) {
680
739
throw new FeatureNotSupportedException ("not supported yet'" );
681
740
} else {
682
- Mutation mutation = (Mutation ) action ;
683
- if (mutation .getFamilyCellMap ().size () != 1 ) {
741
+ Set <byte []> familySet = null ;
742
+ if (action instanceof Get ) {
743
+ Get get = (Get ) action ;
744
+ familySet = get .familySet ();
745
+ } else {
746
+ Mutation mutation = (Mutation ) action ;
747
+ familySet = mutation .getFamilyCellMap ().keySet ();
748
+ }
749
+ if (familySet == null ) {
750
+ throw new ObTableUnexpectedException ("Fail to get family set in action" );
751
+ }
752
+ if (familySet .size () != 1 ) {
684
753
return getTargetTableName (tableNameString );
685
754
} else {
686
- byte [] nextFamily = mutation . getFamilyCellMap (). keySet () .iterator ().next ();
755
+ byte [] nextFamily = familySet .iterator ().next ();
687
756
if (family != null && !Arrays .equals (family , nextFamily )) {
688
757
return getTargetTableName (tableNameString );
689
758
} else if (family == null ) {
@@ -702,6 +771,7 @@ public Object[] batch(List<? extends Row> actions) throws IOException {
702
771
return results ;
703
772
}
704
773
774
+ @ Override
705
775
public <R > void batchCallback (List <? extends Row > actions , Object [] results ,
706
776
Batch .Callback <R > callback ) throws IOException ,
707
777
InterruptedException {
@@ -879,17 +949,21 @@ public Result call() throws IOException {
879
949
@ Override
880
950
public Result [] get (List <Get > gets ) throws IOException {
881
951
Result [] results = new Result [gets .size ()];
882
- List <Future <Result >> futures = new LinkedList <>();
883
- for (int i = 0 ; i < gets .size (); i ++) {
884
- int index = i ;
885
- Future <Result > future = executePool .submit (() -> get (gets .get (index )));
886
- futures .add (future );
887
- }
888
- for (int i = 0 ; i < gets .size (); i ++) {
889
- try {
890
- results [i ] = futures .get (i ).get ();
891
- } catch (Exception e ) {
892
- throw new RuntimeException ("gets occur error. index:{" + i + "}" , e );
952
+ if (ObGlobal .isHBaseBatchGetSupport ()) { // get only supported in BatchSupport version
953
+ batch (gets , results );
954
+ } else {
955
+ List <Future <Result >> futures = new LinkedList <>();
956
+ for (int i = 0 ; i < gets .size (); i ++) {
957
+ int index = i ;
958
+ Future <Result > future = executePool .submit (() -> get (gets .get (index )));
959
+ futures .add (future );
960
+ }
961
+ for (int i = 0 ; i < gets .size (); i ++) {
962
+ try {
963
+ results [i ] = futures .get (i ).get ();
964
+ } catch (Exception e ) {
965
+ throw new RuntimeException ("gets occur error. index:{" + i + "}" , e );
966
+ }
893
967
}
894
968
}
895
969
return results ;
@@ -1668,6 +1742,8 @@ public <R extends Message> void batchCoprocessorService(Descriptors.MethodDescri
1668
1742
@ Override
1669
1743
public void setOperationTimeout (int operationTimeout ) {
1670
1744
this .operationTimeout = operationTimeout ;
1745
+ this .obTableClient .setRuntimeMaxWait (operationTimeout );
1746
+ this .obTableClient .setRuntimeBatchMaxWait (operationTimeout );
1671
1747
this .operationExecuteInPool = this .configuration .getBoolean (
1672
1748
HBASE_CLIENT_OPERATION_EXECUTE_IN_POOL ,
1673
1749
(this .operationTimeout != HConstants .DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT ));
@@ -1994,14 +2070,55 @@ private KeyValue modifyQualifier(KeyValue original, byte[] newQualifier) {
1994
2070
private BatchOperation buildBatchOperation (String tableName , List <? extends Row > actions ,
1995
2071
boolean isTableGroup , List <Integer > resultMapSingleOp )
1996
2072
throws FeatureNotSupportedException ,
1997
- IllegalArgumentException {
2073
+ IllegalArgumentException ,
2074
+ IOException {
1998
2075
BatchOperation batch = obTableClient .batchOperation (tableName );
1999
2076
int posInList = -1 ;
2000
2077
int singleOpResultNum ;
2001
2078
for (Row row : actions ) {
2002
2079
singleOpResultNum = 0 ;
2003
2080
posInList ++;
2004
- if (row instanceof Put ) {
2081
+ if (row instanceof Get ) {
2082
+ if (!ObGlobal .isHBaseBatchGetSupport ()) {
2083
+ throw new FeatureNotSupportedException ("server does not support batch get" );
2084
+ }
2085
+ ++singleOpResultNum ;
2086
+ Get get = (Get ) row ;
2087
+ ObTableQuery obTableQuery ;
2088
+ // In a Get operation in ls batch, we need to determine whether the get is a table-group operation or not,
2089
+ // we handle this by appending the column family to the qualifier on the client side.
2090
+ // The server can then use this information to filter the appropriate column families and qualifiers.
2091
+ if ((get .getFamilyMap ().keySet ().isEmpty ()
2092
+ || get .getFamilyMap ().size () > 1 ) &&
2093
+ !get .getColumnFamilyTimeRange ().isEmpty ()) {
2094
+ throw new FeatureNotSupportedException ("setColumnFamilyTimeRange is only supported in single column family for now" );
2095
+ } else if (get .getFamilyMap ().size () == 1 && !get .getColumnFamilyTimeRange ().isEmpty ()) {
2096
+ byte [] family = get .getFamilyMap ().keySet ().iterator ().next ();
2097
+ Map <byte [], TimeRange > colFamTimeRangeMap = get .getColumnFamilyTimeRange ();
2098
+ if (colFamTimeRangeMap .size () > 1 ) {
2099
+ throw new FeatureNotSupportedException ("setColumnFamilyTimeRange is only supported in single column family for now" );
2100
+ } else if (colFamTimeRangeMap .get (family ) == null ) {
2101
+ throw new IllegalArgumentException ("Get family is not matched in ColumnFamilyTimeRange" );
2102
+ } else {
2103
+ TimeRange tr = colFamTimeRangeMap .get (family );
2104
+ get .setTimeRange (tr .getMin (), tr .getMax ());
2105
+ }
2106
+ }
2107
+ NavigableSet <byte []> columnFilters = new TreeSet <>(Bytes .BYTES_COMPARATOR );
2108
+ // in batch get, we need to carry family in qualifier to server even this get is a single-cf operation
2109
+ // because the entire batch may be a multi-cf batch so do not carry family
2110
+ // family in qualifier helps us to know which table to query
2111
+ processColumnFilters (columnFilters , get .getFamilyMap ());
2112
+ obTableQuery = buildObTableQuery (get , columnFilters );
2113
+ ObTableClientQueryImpl query = new ObTableClientQueryImpl (tableName , obTableQuery , obTableClient );
2114
+ try {
2115
+ query .setRowKey (row (colVal ("K" , Bytes .toString (get .getRow ())), colVal ("Q" , null ), colVal ("T" , null )));
2116
+ } catch (Exception e ) {
2117
+ logger .error ("unexpected error occurs when set row key" , e );
2118
+ throw new IOException (e );
2119
+ }
2120
+ batch .addOperation (query );
2121
+ } else if (row instanceof Put ) {
2005
2122
Put put = (Put ) row ;
2006
2123
if (put .isEmpty ()) {
2007
2124
throw new IllegalArgumentException ("No columns to insert for #"
@@ -2049,7 +2166,7 @@ private BatchOperation buildBatchOperation(String tableName, List<? extends Row>
2049
2166
}
2050
2167
} else {
2051
2168
throw new FeatureNotSupportedException (
2052
- "not supported other type in batch yet,only support put and delete" );
2169
+ "not supported other type in batch yet,only support get, put and delete" );
2053
2170
}
2054
2171
resultMapSingleOp .add (singleOpResultNum );
2055
2172
}
0 commit comments