You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/02/24 10:04:55 UTC
[hbase] branch branch-2.2 updated: HBASE-21945 Maintain the
original order when sending batch request
This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2.2 by this push:
new 2add6dc HBASE-21945 Maintain the original order when sending batch request
2add6dc is described below
commit 2add6dcb53dc9eb90a08373e65c3ba2c89a278c9
Author: zhangduo <zh...@apache.org>
AuthorDate: Fri Feb 22 20:39:00 2019 +0800
HBASE-21945 Maintain the original order when sending batch request
Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
.../hbase/client/AsyncBatchRpcRetryingCaller.java | 7 +++++--
.../hadoop/hbase/client/TestAsyncTableBatch.java | 24 ++++++++++++++++++++++
2 files changed, 29 insertions(+), 2 deletions(-)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
index 4e983e5..b62ba6c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
@@ -255,8 +255,11 @@ class AsyncBatchRpcRetryingCaller<T> {
// multiRequestBuilder will be populated with region actions.
// rowMutationsIndexMap will be non-empty after the call if there is RowMutations in the
// action list.
- RequestConverter.buildNoDataRegionActions(entry.getKey(), entry.getValue().actions, cells,
- multiRequestBuilder, regionActionBuilder, actionBuilder, mutationBuilder, nonceGroup,
+ RequestConverter.buildNoDataRegionActions(entry.getKey(),
+ entry.getValue().actions.stream()
+ .sorted((a1, a2) -> Integer.compare(a1.getOriginalIndex(), a2.getOriginalIndex()))
+ .collect(Collectors.toList()),
+ cells, multiRequestBuilder, regionActionBuilder, actionBuilder, mutationBuilder, nonceGroup,
rowMutationsIndexMap);
}
return multiRequestBuilder.build();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
index 3a7614b..717eb24 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.IOException;
import java.io.UncheckedIOException;
@@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -280,8 +282,30 @@ public class TestAsyncTableBatch {
}
try {
futures.get(SPLIT_KEYS.length - 1).get();
+ fail();
} catch (ExecutionException e) {
assertThat(e.getCause(), instanceOf(RetriesExhaustedException.class));
}
}
+
+ @Test
+ public void testPartialSuccessOnSameRegion() throws InterruptedException, ExecutionException {
+ AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
+ List<CompletableFuture<Object>> futures = table.batch(Arrays.asList(
+ new Put(Bytes.toBytes("put")).addColumn(Bytes.toBytes("not-exists"), CQ,
+ Bytes.toBytes("bad")),
+ new Increment(Bytes.toBytes("inc")).addColumn(FAMILY, CQ, 1),
+ new Put(Bytes.toBytes("put")).addColumn(FAMILY, CQ, Bytes.toBytes("good"))));
+ try {
+ futures.get(0).get();
+ fail();
+ } catch (ExecutionException e) {
+ assertThat(e.getCause(), instanceOf(RetriesExhaustedException.class));
+ assertThat(e.getCause().getCause(), instanceOf(NoSuchColumnFamilyException.class));
+ }
+ assertEquals(1, Bytes.toLong(((Result) futures.get(1).get()).getValue(FAMILY, CQ)));
+ assertTrue(((Result) futures.get(2).get()).isEmpty());
+ assertEquals("good",
+ Bytes.toString(table.get(new Get(Bytes.toBytes("put"))).get().getValue(FAMILY, CQ)));
+ }
}