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/03/02 04:25:59 UTC

[hbase] branch branch-2 updated: HBASE-21976 Deal with RetryImmediatelyException for batching request

This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 8677031  HBASE-21976 Deal with RetryImmediatelyException for batching request
8677031 is described below

commit 8677031707ad9d058689da4eff131ab79f2e6e6e
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Mar 1 18:56:48 2019 +0800

    HBASE-21976 Deal with RetryImmediatelyException for batching request
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../hbase/client/AsyncBatchRpcRetryingCaller.java  |  30 ++++--
 .../TestAsyncTableBatchRetryImmediately.java       | 101 +++++++++++++++++++++
 2 files changed, 123 insertions(+), 8 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 b62ba6c..f9bcf74 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
@@ -44,10 +44,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
+import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RetryImmediatelyException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.MultiResponse.RegionResult;
@@ -267,7 +269,8 @@ class AsyncBatchRpcRetryingCaller<T> {
 
   @SuppressWarnings("unchecked")
   private void onComplete(Action action, RegionRequest regionReq, int tries, ServerName serverName,
-      RegionResult regionResult, List<Action> failedActions, Throwable regionException) {
+      RegionResult regionResult, List<Action> failedActions, Throwable regionException,
+      MutableBoolean retryImmediately) {
     Object result = regionResult.result.getOrDefault(action.getOriginalIndex(), regionException);
     if (result == null) {
       LOG.error("Server " + serverName + " sent us neither result nor exception for row '" +
@@ -283,6 +286,9 @@ class AsyncBatchRpcRetryingCaller<T> {
         failOne(action, tries, error, EnvironmentEdgeManager.currentTime(),
           getExtraContextForError(serverName));
       } else {
+        if (!retryImmediately.booleanValue() && error instanceof RetryImmediatelyException) {
+          retryImmediately.setTrue();
+        }
         failedActions.add(action);
       }
     } else {
@@ -293,17 +299,18 @@ class AsyncBatchRpcRetryingCaller<T> {
   private void onComplete(Map<byte[], RegionRequest> actionsByRegion, int tries,
       ServerName serverName, MultiResponse resp) {
     List<Action> failedActions = new ArrayList<>();
+    MutableBoolean retryImmediately = new MutableBoolean(false);
     actionsByRegion.forEach((rn, regionReq) -> {
       RegionResult regionResult = resp.getResults().get(rn);
       Throwable regionException = resp.getException(rn);
       if (regionResult != null) {
         regionReq.actions.forEach(action -> onComplete(action, regionReq, tries, serverName,
-          regionResult, failedActions, regionException));
+          regionResult, failedActions, regionException, retryImmediately));
       } else {
         Throwable error;
         if (regionException == null) {
-          LOG
-            .error("Server sent us neither results nor exceptions for " + Bytes.toStringBinary(rn));
+          LOG.error("Server sent us neither results nor exceptions for {}",
+            Bytes.toStringBinary(rn));
           error = new RuntimeException("Invalid response");
         } else {
           error = translateException(regionException);
@@ -314,12 +321,15 @@ class AsyncBatchRpcRetryingCaller<T> {
           failAll(regionReq.actions.stream(), tries, error, serverName);
           return;
         }
+        if (!retryImmediately.booleanValue() && error instanceof RetryImmediatelyException) {
+          retryImmediately.setTrue();
+        }
         addError(regionReq.actions, error, serverName);
         failedActions.addAll(regionReq.actions);
       }
     });
     if (!failedActions.isEmpty()) {
-      tryResubmit(failedActions.stream(), tries);
+      tryResubmit(failedActions.stream(), tries, retryImmediately.booleanValue());
     }
   }
 
@@ -391,10 +401,14 @@ class AsyncBatchRpcRetryingCaller<T> {
     List<Action> copiedActions = actionsByRegion.values().stream().flatMap(r -> r.actions.stream())
       .collect(Collectors.toList());
     addError(copiedActions, error, serverName);
-    tryResubmit(copiedActions.stream(), tries);
+    tryResubmit(copiedActions.stream(), tries, error instanceof RetryImmediatelyException);
   }
 
-  private void tryResubmit(Stream<Action> actions, int tries) {
+  private void tryResubmit(Stream<Action> actions, int tries, boolean immediately) {
+    if (immediately) {
+      groupAndSend(actions, tries);
+      return;
+    }
     long delayNs;
     if (operationTimeoutNs > 0) {
       long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
@@ -443,7 +457,7 @@ class AsyncBatchRpcRetryingCaller<T> {
           send(actionsByServer, tries);
         }
         if (!locateFailed.isEmpty()) {
-          tryResubmit(locateFailed.stream(), tries);
+          tryResubmit(locateFailed.stream(), tries, false);
         }
       });
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatchRetryImmediately.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatchRetryImmediately.java
new file mode 100644
index 0000000..57cfbec
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatchRetryImmediately.java
@@ -0,0 +1,101 @@
+/**
+ * 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.client;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestAsyncTableBatchRetryImmediately {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAsyncTableBatchRetryImmediately.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName TABLE_NAME = TableName.valueOf("async");
+
+  private static byte[] FAMILY = Bytes.toBytes("cf");
+
+  private static byte[] QUAL = Bytes.toBytes("cq");
+
+  private static byte[] VALUE_PREFIX = new byte[768];
+
+  private static int COUNT = 1000;
+
+  private static AsyncConnection CONN;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    // disable the debug log to avoid flooding the output
+    LogManager.getLogger(AsyncRegionLocatorHelper.class).setLevel(Level.INFO);
+    UTIL.getConfiguration().setLong(HConstants.HBASE_SERVER_SCANNER_MAX_RESULT_SIZE_KEY, 1024);
+    UTIL.startMiniCluster(1);
+    Table table = UTIL.createTable(TABLE_NAME, FAMILY);
+    UTIL.waitTableAvailable(TABLE_NAME);
+    ThreadLocalRandom.current().nextBytes(VALUE_PREFIX);
+    for (int i = 0; i < COUNT; i++) {
+      table.put(new Put(Bytes.toBytes(i)).addColumn(FAMILY, QUAL,
+        Bytes.add(VALUE_PREFIX, Bytes.toBytes(i))));
+    }
+    CONN = ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    CONN.close();
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() {
+    AsyncTable<?> table = CONN.getTable(TABLE_NAME);
+    // if we do not deal with RetryImmediatelyException, we will timeout here since we need to retry
+    // hundreds times.
+    List<Get> gets = IntStream.range(0, COUNT).mapToObj(i -> new Get(Bytes.toBytes(i)))
+      .collect(Collectors.toList());
+    List<Result> results = table.getAll(gets).join();
+    for (int i = 0; i < COUNT; i++) {
+      byte[] value = results.get(i).getValue(FAMILY, QUAL);
+      assertEquals(VALUE_PREFIX.length + 4, value.length);
+      assertArrayEquals(VALUE_PREFIX, Arrays.copyOf(value, VALUE_PREFIX.length));
+      assertEquals(i, Bytes.toInt(value, VALUE_PREFIX.length));
+    }
+  }
+}