HBASE-21976 Deal with RetryImmediatelyException for batching request
Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
parent
f748e489c5
commit
cbdbe6572b
|
@ -44,10 +44,12 @@ import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
import org.apache.commons.lang3.mutable.MutableBoolean;
|
||||||
import org.apache.hadoop.hbase.CellScannable;
|
import org.apache.hadoop.hbase.CellScannable;
|
||||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionLocation;
|
import org.apache.hadoop.hbase.HRegionLocation;
|
||||||
|
import org.apache.hadoop.hbase.RetryImmediatelyException;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.MultiResponse.RegionResult;
|
import org.apache.hadoop.hbase.client.MultiResponse.RegionResult;
|
||||||
|
@ -267,7 +269,8 @@ class AsyncBatchRpcRetryingCaller<T> {
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private void onComplete(Action action, RegionRequest regionReq, int tries, ServerName serverName,
|
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);
|
Object result = regionResult.result.getOrDefault(action.getOriginalIndex(), regionException);
|
||||||
if (result == null) {
|
if (result == null) {
|
||||||
LOG.error("Server " + serverName + " sent us neither result nor exception for row '" +
|
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(),
|
failOne(action, tries, error, EnvironmentEdgeManager.currentTime(),
|
||||||
getExtraContextForError(serverName));
|
getExtraContextForError(serverName));
|
||||||
} else {
|
} else {
|
||||||
|
if (!retryImmediately.booleanValue() && error instanceof RetryImmediatelyException) {
|
||||||
|
retryImmediately.setTrue();
|
||||||
|
}
|
||||||
failedActions.add(action);
|
failedActions.add(action);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
@ -293,17 +299,18 @@ class AsyncBatchRpcRetryingCaller<T> {
|
||||||
private void onComplete(Map<byte[], RegionRequest> actionsByRegion, int tries,
|
private void onComplete(Map<byte[], RegionRequest> actionsByRegion, int tries,
|
||||||
ServerName serverName, MultiResponse resp) {
|
ServerName serverName, MultiResponse resp) {
|
||||||
List<Action> failedActions = new ArrayList<>();
|
List<Action> failedActions = new ArrayList<>();
|
||||||
|
MutableBoolean retryImmediately = new MutableBoolean(false);
|
||||||
actionsByRegion.forEach((rn, regionReq) -> {
|
actionsByRegion.forEach((rn, regionReq) -> {
|
||||||
RegionResult regionResult = resp.getResults().get(rn);
|
RegionResult regionResult = resp.getResults().get(rn);
|
||||||
Throwable regionException = resp.getException(rn);
|
Throwable regionException = resp.getException(rn);
|
||||||
if (regionResult != null) {
|
if (regionResult != null) {
|
||||||
regionReq.actions.forEach(action -> onComplete(action, regionReq, tries, serverName,
|
regionReq.actions.forEach(action -> onComplete(action, regionReq, tries, serverName,
|
||||||
regionResult, failedActions, regionException));
|
regionResult, failedActions, regionException, retryImmediately));
|
||||||
} else {
|
} else {
|
||||||
Throwable error;
|
Throwable error;
|
||||||
if (regionException == null) {
|
if (regionException == null) {
|
||||||
LOG
|
LOG.error("Server sent us neither results nor exceptions for {}",
|
||||||
.error("Server sent us neither results nor exceptions for " + Bytes.toStringBinary(rn));
|
Bytes.toStringBinary(rn));
|
||||||
error = new RuntimeException("Invalid response");
|
error = new RuntimeException("Invalid response");
|
||||||
} else {
|
} else {
|
||||||
error = translateException(regionException);
|
error = translateException(regionException);
|
||||||
|
@ -314,12 +321,15 @@ class AsyncBatchRpcRetryingCaller<T> {
|
||||||
failAll(regionReq.actions.stream(), tries, error, serverName);
|
failAll(regionReq.actions.stream(), tries, error, serverName);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
if (!retryImmediately.booleanValue() && error instanceof RetryImmediatelyException) {
|
||||||
|
retryImmediately.setTrue();
|
||||||
|
}
|
||||||
addError(regionReq.actions, error, serverName);
|
addError(regionReq.actions, error, serverName);
|
||||||
failedActions.addAll(regionReq.actions);
|
failedActions.addAll(regionReq.actions);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
if (!failedActions.isEmpty()) {
|
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())
|
List<Action> copiedActions = actionsByRegion.values().stream().flatMap(r -> r.actions.stream())
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
addError(copiedActions, error, serverName);
|
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;
|
long delayNs;
|
||||||
if (operationTimeoutNs > 0) {
|
if (operationTimeoutNs > 0) {
|
||||||
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
|
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
|
||||||
|
@ -443,7 +457,7 @@ class AsyncBatchRpcRetryingCaller<T> {
|
||||||
send(actionsByServer, tries);
|
send(actionsByServer, tries);
|
||||||
}
|
}
|
||||||
if (!locateFailed.isEmpty()) {
|
if (!locateFailed.isEmpty()) {
|
||||||
tryResubmit(locateFailed.stream(), tries);
|
tryResubmit(locateFailed.stream(), tries, false);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
|
@ -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));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue