Skip to content

Commit f405990

Browse files
authored
HBASE-25678 Support nonce operations for Increment/Append in RowMutations and CheckAndMutate (#3064)
Signed-off-by: stack <[email protected]>
1 parent cc6c14a commit f405990

File tree

7 files changed

+438
-144
lines changed

7 files changed

+438
-144
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java

Lines changed: 21 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -172,7 +172,7 @@ public AsyncBatchRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn,
172172
} else {
173173
action = new Action(rawAction, i);
174174
}
175-
if (rawAction instanceof Append || rawAction instanceof Increment) {
175+
if (hasIncrementOrAppend(rawAction)) {
176176
action.setNonce(conn.getNonceGenerator().newNonce());
177177
}
178178
this.actions.add(action);
@@ -184,6 +184,26 @@ public AsyncBatchRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn,
184184
this.startNs = System.nanoTime();
185185
}
186186

187+
private static boolean hasIncrementOrAppend(Row action) {
188+
if (action instanceof Append || action instanceof Increment) {
189+
return true;
190+
} else if (action instanceof RowMutations) {
191+
return hasIncrementOrAppend((RowMutations) action);
192+
} else if (action instanceof CheckAndMutate) {
193+
return hasIncrementOrAppend(((CheckAndMutate) action).getAction());
194+
}
195+
return false;
196+
}
197+
198+
private static boolean hasIncrementOrAppend(RowMutations mutations) {
199+
for (Mutation mutation : mutations.getMutations()) {
200+
if (mutation instanceof Append || mutation instanceof Increment) {
201+
return true;
202+
}
203+
}
204+
return false;
205+
}
206+
187207
private long remainingTimeNs() {
188208
return operationTimeoutNs - (System.nanoTime() - startNs);
189209
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java

Lines changed: 21 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -66,7 +66,6 @@
6666
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
6767
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
6868
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
69-
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
7069

7170
/**
7271
* The implementation of RawAsyncTable.
@@ -324,7 +323,7 @@ public CompletableFuture<Boolean> thenPut(Put put) {
324323
.action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc,
325324
stub, put,
326325
(rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
327-
null, timeRange, p),
326+
null, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),
328327
(c, r) -> r.getProcessed()))
329328
.call();
330329
}
@@ -336,7 +335,7 @@ public CompletableFuture<Boolean> thenDelete(Delete delete) {
336335
.action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller,
337336
loc, stub, delete,
338337
(rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
339-
null, timeRange, d),
338+
null, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),
340339
(c, r) -> r.getProcessed()))
341340
.call();
342341
}
@@ -349,8 +348,9 @@ public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
349348
rpcTimeoutNs)
350349
.action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller,
351350
loc, stub, mutation,
352-
(rn, rm) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
353-
null, timeRange, rm), CheckAndMutateResult::isSuccess))
351+
(rn, rm) -> RequestConverter.buildMultiRequest(rn, row, family, qualifier, op, value,
352+
null, timeRange, rm, HConstants.NO_NONCE, HConstants.NO_NONCE),
353+
CheckAndMutateResult::isSuccess))
354354
.call();
355355
}
356356
}
@@ -387,7 +387,7 @@ public CompletableFuture<Boolean> thenPut(Put put) {
387387
.action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc,
388388
stub, put,
389389
(rn, p) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null,
390-
filter, timeRange, p),
390+
filter, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),
391391
(c, r) -> r.getProcessed()))
392392
.call();
393393
}
@@ -398,7 +398,7 @@ public CompletableFuture<Boolean> thenDelete(Delete delete) {
398398
.action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller,
399399
loc, stub, delete,
400400
(rn, d) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null,
401-
filter, timeRange, d),
401+
filter, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),
402402
(c, r) -> r.getProcessed()))
403403
.call();
404404
}
@@ -410,8 +410,9 @@ public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
410410
rpcTimeoutNs)
411411
.action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller,
412412
loc, stub, mutation,
413-
(rn, rm) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null,
414-
filter, timeRange, rm), CheckAndMutateResult::isSuccess))
413+
(rn, rm) -> RequestConverter.buildMultiRequest(rn, row, null, null, null, null,
414+
filter, timeRange, rm, HConstants.NO_NONCE, HConstants.NO_NONCE),
415+
CheckAndMutateResult::isSuccess))
415416
.call();
416417
}
417418
}
@@ -430,28 +431,32 @@ public CompletableFuture<CheckAndMutateResult> checkAndMutate(CheckAndMutate che
430431
if (mutation instanceof Put) {
431432
validatePut((Put) mutation, conn.connConf.getMaxKeyValueSize());
432433
}
434+
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
435+
long nonce = conn.getNonceGenerator().newNonce();
433436
return RawAsyncTableImpl.this.<CheckAndMutateResult> newCaller(checkAndMutate.getRow(),
434437
mutation.getPriority(), rpcTimeoutNs)
435438
.action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller,
436439
loc, stub, mutation,
437440
(rn, m) -> RequestConverter.buildMutateRequest(rn, checkAndMutate.getRow(),
438441
checkAndMutate.getFamily(), checkAndMutate.getQualifier(),
439442
checkAndMutate.getCompareOp(), checkAndMutate.getValue(), checkAndMutate.getFilter(),
440-
checkAndMutate.getTimeRange(), m),
443+
checkAndMutate.getTimeRange(), m, nonceGroup, nonce),
441444
(c, r) -> ResponseConverter.getCheckAndMutateResult(r, c.cellScanner())))
442445
.call();
443446
} else if (checkAndMutate.getAction() instanceof RowMutations) {
444447
RowMutations rowMutations = (RowMutations) checkAndMutate.getAction();
445448
validatePutsInRowMutations(rowMutations, conn.connConf.getMaxKeyValueSize());
449+
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
450+
long nonce = conn.getNonceGenerator().newNonce();
446451
return RawAsyncTableImpl.this.<CheckAndMutateResult> newCaller(checkAndMutate.getRow(),
447452
rowMutations.getMaxPriority(), rpcTimeoutNs)
448453
.action((controller, loc, stub) ->
449454
RawAsyncTableImpl.this.<CheckAndMutateResult, CheckAndMutateResult> mutateRow(
450455
controller, loc, stub, rowMutations,
451-
(rn, rm) -> RequestConverter.buildMutateRequest(rn, checkAndMutate.getRow(),
456+
(rn, rm) -> RequestConverter.buildMultiRequest(rn, checkAndMutate.getRow(),
452457
checkAndMutate.getFamily(), checkAndMutate.getQualifier(),
453458
checkAndMutate.getCompareOp(), checkAndMutate.getValue(), checkAndMutate.getFilter(),
454-
checkAndMutate.getTimeRange(), rm),
459+
checkAndMutate.getTimeRange(), rm, nonceGroup, nonce),
455460
resp -> resp))
456461
.call();
457462
} else {
@@ -516,16 +521,13 @@ public void run(MultiResponse resp) {
516521
@Override
517522
public CompletableFuture<Result> mutateRow(RowMutations mutations) {
518523
validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
524+
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
525+
long nonce = conn.getNonceGenerator().newNonce();
519526
return this.<Result> newCaller(mutations.getRow(), mutations.getMaxPriority(),
520527
writeRpcTimeoutNs).action((controller, loc, stub) ->
521528
this.<Result, Result> mutateRow(controller, loc, stub, mutations,
522-
(rn, rm) -> {
523-
RegionAction.Builder regionMutationBuilder = RequestConverter
524-
.buildRegionAction(rn, rm);
525-
regionMutationBuilder.setAtomic(true);
526-
return MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build())
527-
.build();
528-
}, resp -> resp))
529+
(rn, rm) -> RequestConverter.buildMultiRequest(rn, rm, nonceGroup, nonce),
530+
resp -> resp))
529531
.call();
530532
}
531533

hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java

Lines changed: 85 additions & 53 deletions
Original file line numberDiff line numberDiff line change
@@ -203,39 +203,78 @@ public static GetRequest buildGetRequest(final byte[] regionName,
203203
*/
204204
public static MutateRequest buildMutateRequest(final byte[] regionName, final byte[] row,
205205
final byte[] family, final byte[] qualifier, final CompareOperator op, final byte[] value,
206-
final Filter filter, final TimeRange timeRange, final Mutation mutation) throws IOException {
207-
return MutateRequest.newBuilder()
208-
.setRegion(buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName))
209-
.setMutation(ProtobufUtil.toMutation(getMutationType(mutation), mutation))
206+
final Filter filter, final TimeRange timeRange, final Mutation mutation, long nonceGroup,
207+
long nonce) throws IOException {
208+
MutateRequest.Builder builder = MutateRequest.newBuilder();
209+
if (mutation instanceof Increment || mutation instanceof Append) {
210+
builder.setMutation(ProtobufUtil.toMutation(getMutationType(mutation), mutation, nonce))
211+
.setNonceGroup(nonceGroup);
212+
} else {
213+
builder.setMutation(ProtobufUtil.toMutation(getMutationType(mutation), mutation));
214+
}
215+
return builder.setRegion(buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName))
210216
.setCondition(buildCondition(row, family, qualifier, op, value, filter, timeRange))
211217
.build();
212218
}
213219

214220
/**
215-
* Create a protocol buffer MutateRequest for conditioned row mutations
221+
* Create a protocol buffer MultiRequest for conditioned row mutations
216222
*
217-
* @return a mutate request
223+
* @return a multi request
218224
* @throws IOException
219225
*/
220-
public static ClientProtos.MultiRequest buildMutateRequest(final byte[] regionName,
226+
public static ClientProtos.MultiRequest buildMultiRequest(final byte[] regionName,
221227
final byte[] row, final byte[] family, final byte[] qualifier,
222228
final CompareOperator op, final byte[] value, final Filter filter, final TimeRange timeRange,
223-
final RowMutations rowMutations) throws IOException {
229+
final RowMutations rowMutations, long nonceGroup, long nonce) throws IOException {
230+
return buildMultiRequest(regionName, rowMutations, buildCondition(row, family, qualifier, op,
231+
value, filter, timeRange), nonceGroup, nonce);
232+
}
233+
234+
/**
235+
* Create a protocol buffer MultiRequest for row mutations
236+
*
237+
* @return a multi request
238+
*/
239+
public static ClientProtos.MultiRequest buildMultiRequest(final byte[] regionName,
240+
final RowMutations rowMutations, long nonceGroup, long nonce) throws IOException {
241+
return buildMultiRequest(regionName, rowMutations, null, nonceGroup, nonce);
242+
}
243+
244+
private static ClientProtos.MultiRequest buildMultiRequest(final byte[] regionName,
245+
final RowMutations rowMutations, final Condition condition, long nonceGroup, long nonce)
246+
throws IOException {
224247
RegionAction.Builder builder =
225-
getRegionActionBuilderWithRegion(RegionAction.newBuilder(), regionName);
248+
getRegionActionBuilderWithRegion(RegionAction.newBuilder(), regionName);
226249
builder.setAtomic(true);
250+
251+
boolean hasNonce = false;
227252
ClientProtos.Action.Builder actionBuilder = ClientProtos.Action.newBuilder();
228253
MutationProto.Builder mutationBuilder = MutationProto.newBuilder();
229254
for (Mutation mutation: rowMutations.getMutations()) {
230255
mutationBuilder.clear();
231-
MutationProto mp = ProtobufUtil.toMutation(getMutationType(mutation), mutation,
232-
mutationBuilder);
256+
MutationProto mp;
257+
if (mutation instanceof Increment || mutation instanceof Append) {
258+
mp = ProtobufUtil.toMutation(getMutationType(mutation), mutation, mutationBuilder, nonce);
259+
hasNonce = true;
260+
} else {
261+
mp = ProtobufUtil.toMutation(getMutationType(mutation), mutation, mutationBuilder);
262+
}
233263
actionBuilder.clear();
234264
actionBuilder.setMutation(mp);
235265
builder.addAction(actionBuilder.build());
236266
}
237-
return ClientProtos.MultiRequest.newBuilder().addRegionAction(builder.setCondition(
238-
buildCondition(row, family, qualifier, op, value, filter, timeRange)).build()).build();
267+
268+
if (condition != null) {
269+
builder.setCondition(condition);
270+
}
271+
272+
MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder();
273+
if (hasNonce) {
274+
multiRequestBuilder.setNonceGroup(nonceGroup);
275+
}
276+
277+
return multiRequestBuilder.addRegionAction(builder.build()).build();
239278
}
240279

241280
/**
@@ -318,33 +357,6 @@ public static MutateRequest buildMutateRequest(
318357
return builder.build();
319358
}
320359

321-
/**
322-
* Create a protocol buffer MultiRequest for row mutations.
323-
* Does not propagate Action absolute position. Does not set atomic action on the created
324-
* RegionAtomic. Caller should do that if wanted.
325-
* @param regionName
326-
* @param rowMutations
327-
* @return a data-laden RegionMutation.Builder
328-
* @throws IOException
329-
*/
330-
public static RegionAction.Builder buildRegionAction(final byte [] regionName,
331-
final RowMutations rowMutations)
332-
throws IOException {
333-
RegionAction.Builder builder =
334-
getRegionActionBuilderWithRegion(RegionAction.newBuilder(), regionName);
335-
ClientProtos.Action.Builder actionBuilder = ClientProtos.Action.newBuilder();
336-
MutationProto.Builder mutationBuilder = MutationProto.newBuilder();
337-
for (Mutation mutation: rowMutations.getMutations()) {
338-
mutationBuilder.clear();
339-
MutationProto mp = ProtobufUtil.toMutation(getMutationType(mutation), mutation,
340-
mutationBuilder);
341-
actionBuilder.clear();
342-
actionBuilder.setMutation(mp);
343-
builder.addAction(actionBuilder.build());
344-
}
345-
return builder;
346-
}
347-
348360
public static RegionAction.Builder getRegionActionBuilderWithRegion(
349361
final RegionAction.Builder regionActionBuilder, final byte [] regionName) {
350362
RegionSpecifier region = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
@@ -562,9 +574,6 @@ public static void buildNoDataRegionActions(final byte[] regionName,
562574
throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
563575
}
564576
}
565-
if (!multiRequestBuilder.hasNonceGroup() && hasNonce) {
566-
multiRequestBuilder.setNonceGroup(nonceGroup);
567-
}
568577
if (builder.getActionCount() > 0) {
569578
multiRequestBuilder.addRegionAction(builder.build());
570579
}
@@ -578,8 +587,11 @@ public static void buildNoDataRegionActions(final byte[] regionName,
578587
builder.clear();
579588
getRegionActionBuilderWithRegion(builder, regionName);
580589

581-
buildNoDataRegionAction((RowMutations) action.getAction(), cells, builder, actionBuilder,
582-
mutationBuilder);
590+
boolean hasIncrementOrAppend = buildNoDataRegionAction((RowMutations) action.getAction(),
591+
cells, action.getNonce(), builder, actionBuilder, mutationBuilder);
592+
if (hasIncrementOrAppend) {
593+
hasNonce = true;
594+
}
583595
builder.setAtomic(true);
584596

585597
multiRequestBuilder.addRegionAction(builder.build());
@@ -613,16 +625,21 @@ public static void buildNoDataRegionActions(final byte[] regionName,
613625
} else if (cam.getAction() instanceof Increment) {
614626
actionBuilder.clear();
615627
mutationBuilder.clear();
616-
buildNoDataRegionAction((Increment) cam.getAction(), cells, HConstants.NO_NONCE, builder,
628+
buildNoDataRegionAction((Increment) cam.getAction(), cells, action.getNonce(), builder,
617629
actionBuilder, mutationBuilder);
630+
hasNonce = true;
618631
} else if (cam.getAction() instanceof Append) {
619632
actionBuilder.clear();
620633
mutationBuilder.clear();
621-
buildNoDataRegionAction((Append) cam.getAction(), cells, HConstants.NO_NONCE, builder,
634+
buildNoDataRegionAction((Append) cam.getAction(), cells, action.getNonce(), builder,
622635
actionBuilder, mutationBuilder);
636+
hasNonce = true;
623637
} else if (cam.getAction() instanceof RowMutations) {
624-
buildNoDataRegionAction((RowMutations) cam.getAction(), cells, builder, actionBuilder,
625-
mutationBuilder);
638+
boolean hasIncrementOrAppend = buildNoDataRegionAction((RowMutations) cam.getAction(),
639+
cells, action.getNonce(), builder, actionBuilder, mutationBuilder);
640+
if (hasIncrementOrAppend) {
641+
hasNonce = true;
642+
}
626643
builder.setAtomic(true);
627644
} else {
628645
throw new DoNotRetryIOException("CheckAndMutate doesn't support " +
@@ -635,6 +652,10 @@ public static void buildNoDataRegionActions(final byte[] regionName,
635652
// in the overall multiRequest.
636653
indexMap.put(multiRequestBuilder.getRegionActionCount() - 1, action.getOriginalIndex());
637654
}
655+
656+
if (!multiRequestBuilder.hasNonceGroup() && hasNonce) {
657+
multiRequestBuilder.setNonceGroup(nonceGroup);
658+
}
638659
}
639660

640661
private static void buildNoDataRegionAction(final Put put, final List<CellScannable> cells,
@@ -684,18 +705,29 @@ private static void buildNoDataRegionAction(final Append append,
684705
MutationType.APPEND, append, mutationBuilder, nonce)));
685706
}
686707

687-
private static void buildNoDataRegionAction(final RowMutations rowMutations,
688-
final List<CellScannable> cells, final RegionAction.Builder regionActionBuilder,
708+
/**
709+
* @return whether or not the rowMutations has a Increment or Append
710+
*/
711+
private static boolean buildNoDataRegionAction(final RowMutations rowMutations,
712+
final List<CellScannable> cells, long nonce, final RegionAction.Builder regionActionBuilder,
689713
final ClientProtos.Action.Builder actionBuilder, final MutationProto.Builder mutationBuilder)
690714
throws IOException {
715+
boolean ret = false;
691716
for (Mutation mutation: rowMutations.getMutations()) {
692717
mutationBuilder.clear();
693-
MutationProto mp = ProtobufUtil.toMutationNoData(getMutationType(mutation), mutation,
694-
mutationBuilder);
718+
MutationProto mp;
719+
if (mutation instanceof Increment || mutation instanceof Append) {
720+
mp = ProtobufUtil.toMutationNoData(getMutationType(mutation), mutation, mutationBuilder,
721+
nonce);
722+
ret = true;
723+
} else {
724+
mp = ProtobufUtil.toMutationNoData(getMutationType(mutation), mutation, mutationBuilder);
725+
}
695726
cells.add(mutation);
696727
actionBuilder.clear();
697728
regionActionBuilder.addAction(actionBuilder.setMutation(mp).build());
698729
}
730+
return ret;
699731
}
700732

701733
private static MutationType getMutationType(Mutation mutation) {

hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1551,7 +1551,7 @@ default List<Pair<Cell, Cell>> postAppendBeforeWAL(
15511551
List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size());
15521552
for (Pair<Cell, Cell> pair : cellPairs) {
15531553
resultPairs.add(new Pair<>(pair.getFirst(),
1554-
postMutationBeforeWAL(ctx, MutationType.INCREMENT, mutation, pair.getFirst(),
1554+
postMutationBeforeWAL(ctx, MutationType.APPEND, mutation, pair.getFirst(),
15551555
pair.getSecond())));
15561556
}
15571557
return resultPairs;

0 commit comments

Comments
 (0)