Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
Expand All @@ -56,6 +57,7 @@
import org.apache.hadoop.hbase.client.MultiResponse.RegionResult;
import org.apache.hadoop.hbase.client.RetriesExhaustedException.ThrowableWithExtraContext;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
import org.apache.hadoop.hbase.client.backoff.HBaseServerExceptionPauseManager;
import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.util.Bytes;
Expand Down Expand Up @@ -102,10 +104,6 @@ class AsyncBatchRpcRetryingCaller<T> {

private final IdentityHashMap<Action, List<ThrowableWithExtraContext>> action2Errors;

private final long pauseNs;

private final long pauseNsForServerOverloaded;

private final int maxAttempts;

private final long operationTimeoutNs;
Expand All @@ -116,6 +114,8 @@ class AsyncBatchRpcRetryingCaller<T> {

private final long startNs;

private final HBaseServerExceptionPauseManager pauseManager;

// we can not use HRegionLocation as the map key because the hashCode and equals method of
// HRegionLocation only consider serverName.
private static final class RegionRequest {
Expand Down Expand Up @@ -155,15 +155,14 @@ public AsyncBatchRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn,
this.retryTimer = retryTimer;
this.conn = conn;
this.tableName = tableName;
this.pauseNs = pauseNs;
this.pauseNsForServerOverloaded = pauseNsForServerOverloaded;
this.maxAttempts = maxAttempts;
this.operationTimeoutNs = operationTimeoutNs;
this.rpcTimeoutNs = rpcTimeoutNs;
this.startLogErrorsCnt = startLogErrorsCnt;
this.actions = new ArrayList<>(actions.size());
this.futures = new ArrayList<>(actions.size());
this.action2Future = new IdentityHashMap<>(actions.size());
this.pauseManager = new HBaseServerExceptionPauseManager(pauseNs, pauseNsForServerOverloaded);
for (int i = 0, n = actions.size(); i < n; i++) {
Row rawAction = actions.get(i);
Action action;
Expand Down Expand Up @@ -360,7 +359,7 @@ private void onComplete(Map<byte[], RegionRequest> actionsByRegion, int tries,
}
});
if (!failedActions.isEmpty()) {
tryResubmit(failedActions.stream(), tries, retryImmediately.booleanValue(), false);
tryResubmit(failedActions.stream(), tries, retryImmediately.booleanValue(), null);
}
}

Expand Down Expand Up @@ -465,18 +464,25 @@ private void onError(Map<byte[], RegionRequest> actionsByRegion, int tries, Thro
List<Action> copiedActions = actionsByRegion.values().stream().flatMap(r -> r.actions.stream())
.collect(Collectors.toList());
addError(copiedActions, error, serverName);
tryResubmit(copiedActions.stream(), tries, error instanceof RetryImmediatelyException,
HBaseServerException.isServerOverloaded(error));
tryResubmit(copiedActions.stream(), tries, error instanceof RetryImmediatelyException, error);
}

private void tryResubmit(Stream<Action> actions, int tries, boolean immediately,
boolean isServerOverloaded) {
Throwable error) {
if (immediately) {
groupAndSend(actions, tries);
return;
}
long delayNs;
long pauseNsToUse = isServerOverloaded ? pauseNsForServerOverloaded : pauseNs;
boolean isServerOverloaded = HBaseServerException.isServerOverloaded(error);
OptionalLong maybePauseNsToUse =
pauseManager.getPauseNsFromException(error, remainingTimeNs() - SLEEP_DELTA_NS);
if (!maybePauseNsToUse.isPresent()) {
failAll(actions, tries);
return;
}
long pauseNsToUse = maybePauseNsToUse.getAsLong();

if (operationTimeoutNs > 0) {
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
if (maxDelayNs <= 0) {
Expand Down Expand Up @@ -528,7 +534,7 @@ private void groupAndSend(Stream<Action> actions, int tries) {
sendOrDelay(actionsByServer, tries);
}
if (!locateFailed.isEmpty()) {
tryResubmit(locateFailed.stream(), tries, false, false);
tryResubmit(locateFailed.stream(), tries, false, null);
}
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
Expand All @@ -35,6 +36,7 @@
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.backoff.HBaseServerExceptionPauseManager;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
Expand All @@ -56,10 +58,6 @@ public abstract class AsyncRpcRetryingCaller<T> {

private final long startNs;

private final long pauseNs;

private final long pauseNsForServerOverloaded;

private int tries = 1;

private final int maxAttempts;
Expand All @@ -78,14 +76,14 @@ public abstract class AsyncRpcRetryingCaller<T> {

protected final HBaseRpcController controller;

private final HBaseServerExceptionPauseManager pauseManager;

public AsyncRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, int priority,
long pauseNs, long pauseNsForServerOverloaded, int maxAttempts, long operationTimeoutNs,
long rpcTimeoutNs, int startLogErrorsCnt) {
this.retryTimer = retryTimer;
this.conn = conn;
this.priority = priority;
this.pauseNs = pauseNs;
this.pauseNsForServerOverloaded = pauseNsForServerOverloaded;
this.maxAttempts = maxAttempts;
this.operationTimeoutNs = operationTimeoutNs;
this.rpcTimeoutNs = rpcTimeoutNs;
Expand All @@ -95,6 +93,7 @@ public AsyncRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, int pr
this.controller.setPriority(priority);
this.exceptions = new ArrayList<>();
this.startNs = System.nanoTime();
this.pauseManager = new HBaseServerExceptionPauseManager(pauseNs, pauseNsForServerOverloaded);
}

private long elapsedMs() {
Expand Down Expand Up @@ -125,8 +124,14 @@ protected final void resetCallTimeout() {
}

private void tryScheduleRetry(Throwable error) {
long pauseNsToUse =
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;
OptionalLong maybePauseNsToUse =
pauseManager.getPauseNsFromException(error, remainingTimeNs() - SLEEP_DELTA_NS);
if (!maybePauseNsToUse.isPresent()) {
completeExceptionally();
return;
}
long pauseNsToUse = maybePauseNsToUse.getAsLong();

long delayNs;
if (operationTimeoutNs > 0) {
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.DoNotRetryIOException;
Expand All @@ -43,6 +44,7 @@
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer.ScanResumer;
import org.apache.hadoop.hbase.client.backoff.HBaseServerExceptionPauseManager;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
Expand Down Expand Up @@ -99,10 +101,6 @@ class AsyncScanSingleRegionRpcRetryingCaller {

private final long scannerLeaseTimeoutPeriodNs;

private final long pauseNs;

private final long pauseNsForServerOverloaded;

private final int maxAttempts;

private final long scanTimeoutNs;
Expand Down Expand Up @@ -131,6 +129,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {

private long nextCallSeq = -1L;

private final HBaseServerExceptionPauseManager pauseManager;

private enum ScanControllerState {
INITIALIZED,
SUSPENDED,
Expand Down Expand Up @@ -330,8 +330,6 @@ public AsyncScanSingleRegionRpcRetryingCaller(Timer retryTimer, AsyncConnectionI
this.loc = loc;
this.regionServerRemote = isRegionServerRemote;
this.scannerLeaseTimeoutPeriodNs = scannerLeaseTimeoutPeriodNs;
this.pauseNs = pauseNs;
this.pauseNsForServerOverloaded = pauseNsForServerOverloaded;
this.maxAttempts = maxAttempts;
this.scanTimeoutNs = scanTimeoutNs;
this.rpcTimeoutNs = rpcTimeoutNs;
Expand All @@ -346,6 +344,7 @@ public AsyncScanSingleRegionRpcRetryingCaller(Timer retryTimer, AsyncConnectionI
this.controller = conn.rpcControllerFactory.newController();
this.controller.setPriority(priority);
this.exceptions = new ArrayList<>();
this.pauseManager = new HBaseServerExceptionPauseManager(pauseNs, pauseNsForServerOverloaded);
}

private long elapsedMs() {
Expand Down Expand Up @@ -419,8 +418,15 @@ private void onError(Throwable error) {
return;
}
long delayNs;
long pauseNsToUse =
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;

OptionalLong maybePauseNsToUse =
pauseManager.getPauseNsFromException(error, remainingTimeNs() - SLEEP_DELTA_NS);
if (!maybePauseNsToUse.isPresent()) {
completeExceptionally(!scannerClosed);
return;
}
long pauseNsToUse = maybePauseNsToUse.getAsLong();

if (scanTimeoutNs > 0) {
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
if (maxDelayNs <= 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseServerException;
import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException;
import org.apache.hadoop.hbase.quotas.RpcThrottlingException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.ipc.RemoteException;
Expand Down Expand Up @@ -140,14 +141,28 @@ public T callWithRetries(RetryingCallable<T> callable, int callTimeout)
if (tries >= maxAttempts - 1) {
throw new RetriesExhaustedException(tries, exceptions);
}
// If the server is dead, we need to wait a little before retrying, to give
// a chance to the regions to be moved
// get right pause time, start by RETRY_BACKOFF[0] * pauseBase, where pauseBase might be
// special when encountering an exception indicating the server is overloaded.
// see #HBASE-17114 and HBASE-26807
long pauseBase =
HBaseServerException.isServerOverloaded(t) ? pauseForServerOverloaded : pause;
expectedSleep = callable.sleep(pauseBase, tries);

if (t instanceof RpcThrottlingException) {
RpcThrottlingException rpcThrottlingException = (RpcThrottlingException) t;
expectedSleep = rpcThrottlingException.getWaitInterval();
if (LOG.isDebugEnabled()) {
LOG.debug("Sleeping for {}ms after catching RpcThrottlingException", expectedSleep,
rpcThrottlingException);
}
} else {
expectedSleep =
HBaseServerException.isServerOverloaded(t) ? pauseForServerOverloaded : pause;

// only factor in retry adjustment for non-RpcThrottlingExceptions
// because RpcThrottlingExceptions tell you how long to wait

// If the server is dead, we need to wait a little before retrying, to give
// a chance to the regions to be moved
// get right pause time, start by RETRY_BACKOFF[0] * pauseBase, where pauseBase might be
// special when encountering an exception indicating the server is overloaded.
// see #HBASE-17114 and HBASE-26807
expectedSleep = callable.sleep(expectedSleep, tries);
}

// If, after the planned sleep, there won't be enough time left, we stop now.
long duration = singleCallDuration(expectedSleep);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.backoff;

import java.util.OptionalLong;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.HBaseServerException;
import org.apache.hadoop.hbase.quotas.RpcThrottlingException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@InterfaceAudience.Private
public class HBaseServerExceptionPauseManager {
private static final Logger LOG = LoggerFactory.getLogger(HBaseServerExceptionPauseManager.class);

private final long pauseNs;
private final long pauseNsForServerOverloaded;

public HBaseServerExceptionPauseManager(long pauseNs, long pauseNsForServerOverloaded) {
this.pauseNs = pauseNs;
this.pauseNsForServerOverloaded = pauseNsForServerOverloaded;
}

public OptionalLong getPauseNsFromException(Throwable error, long remainingTimeNs) {
long expectedSleepNs;
if (error instanceof RpcThrottlingException) {
RpcThrottlingException rpcThrottlingException = (RpcThrottlingException) error;
expectedSleepNs = TimeUnit.MILLISECONDS.toNanos(rpcThrottlingException.getWaitInterval());
if (expectedSleepNs > remainingTimeNs) {
return OptionalLong.empty();
}
if (LOG.isDebugEnabled()) {
LOG.debug("Sleeping for {}ms after catching RpcThrottlingException", expectedSleepNs,
rpcThrottlingException);
}
} else {
expectedSleepNs =
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;
}
return OptionalLong.of(expectedSleepNs);
}

}
Loading