Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 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,8 +57,10 @@
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.quotas.RpcThrottlingException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
Expand Down Expand Up @@ -102,10 +105,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 +115,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 +156,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 +360,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,27 +465,40 @@ 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;
}

boolean isServerOverloaded = HBaseServerException.isServerOverloaded(error);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to test this here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe this logic is covered by existing tests in TestAsyncClientPauseForServerOverloaded

OptionalLong maybePauseNsToUse =
pauseManager.getPauseNsFromException(error, remainingTimeNs() - SLEEP_DELTA_NS);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's just pass the tries to this method so it could return the delayNs directly? The instanceof below seems strange...

if (!maybePauseNsToUse.isPresent()) {
failAll(actions, tries);
return;
}
long pauseNsToUse = maybePauseNsToUse.getAsLong();
if (!(error instanceof RpcThrottlingException)) {
// RpcThrottlingException tells us exactly how long the client should wait for,
// so we should not factor in the retry count for said exception
pauseNsToUse = getPauseTime(pauseNsToUse, tries - 1);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this meas the pauseNsToUse is just delayNs now?

Copy link
Contributor Author

@rmdmattingly rmdmattingly Jun 21, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The small distinction is that they may still differ if the requested pause is longer than the operationTimeout allows. I've also pushed that logic to the PauseManager class

}

long delayNs;
long pauseNsToUse = isServerOverloaded ? pauseNsForServerOverloaded : pauseNs;
if (operationTimeoutNs > 0) {
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
if (maxDelayNs <= 0) {
failAll(actions, tries);
return;
}
delayNs = Math.min(maxDelayNs, getPauseTime(pauseNsToUse, tries - 1));
delayNs = Math.min(maxDelayNs, pauseNsToUse);
} else {
delayNs = getPauseTime(pauseNsToUse, tries - 1);
delayNs = pauseNsToUse;
}

if (isServerOverloaded) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems we only need isServerOverloaded here?

Expand Down Expand Up @@ -529,7 +542,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,8 +36,10 @@
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.quotas.RpcThrottlingException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FutureUtils;
import org.apache.yetus.audience.InterfaceAudience;
Expand All @@ -56,10 +59,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 +77,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 +94,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,19 +125,32 @@ 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();

if (!(error instanceof RpcThrottlingException)) {
// RpcThrottlingException tells us exactly how long the client should wait for,
// so we should not factor in the retry count for said exception
pauseNsToUse = getPauseTime(pauseNsToUse, tries - 1);
}

long delayNs;
if (operationTimeoutNs > 0) {
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
if (maxDelayNs <= 0) {
completeExceptionally();
return;
}
delayNs = Math.min(maxDelayNs, getPauseTime(pauseNsToUse, tries - 1));
delayNs = Math.min(maxDelayNs, pauseNsToUse);
} else {
delayNs = getPauseTime(pauseNsToUse, tries - 1);
delayNs = pauseNsToUse;
}

tries++;
if (HBaseServerException.isServerOverloaded(error)) {
Optional<MetricsConnection> metrics = conn.getConnectionMetrics();
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,10 +44,12 @@
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;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.quotas.RpcThrottlingException;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
Expand Down Expand Up @@ -99,10 +102,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 +130,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {

private long nextCallSeq = -1L;

private final HBaseServerExceptionPauseManager pauseManager;

private enum ScanControllerState {
INITIALIZED,
SUSPENDED,
Expand Down Expand Up @@ -330,8 +331,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 +345,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 @@ -418,19 +418,33 @@ private void onError(Throwable error) {
completeExceptionally(!scannerClosed);
return;
}

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

if (!(error instanceof RpcThrottlingException)) {
// RpcThrottlingException tells us exactly how long the client should wait for,
// so we should not factor in the retry count for said exception
pauseNsToUse = getPauseTime(pauseNsToUse, tries - 1);
}

long delayNs;
long pauseNsToUse =
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;
if (scanTimeoutNs > 0) {
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
if (maxDelayNs <= 0) {
completeExceptionally(!scannerClosed);
return;
}
delayNs = Math.min(maxDelayNs, getPauseTime(pauseNsToUse, tries - 1));
delayNs = Math.min(maxDelayNs, pauseNsToUse);
} else {
delayNs = getPauseTime(pauseNsToUse, tries - 1);
delayNs = pauseNsToUse;
}

if (scannerClosed) {
completeWhenError(false);
return;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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;
}

/**
* Returns the nanos, if any, for which the client should wait
* @param error The exception from the server
* @param remainingTimeNs The remaining nanos before timeout
* @return The time, in nanos, to pause. If empty then pausing would exceed our timeout, so we
* should throw now
*/
public OptionalLong getPauseNsFromException(Throwable error, long remainingTimeNs) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add javadoc here to describe the meaning of the return value. For me, I think maybe return long directly is a better choice, as we could return -1 if we should fail. Returning OptionalLong seems indicating that, if we return OptionalLong.empty, the upper layer should decide the pauseNs by their own?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll definitely add a javadoc. Regarding a normal long and -1 to represent exceeding the timeout, I see what you mean. But I like that the empty Optional case is tightly controlled here — there's no opportunity for other code to decide to use -1 arbitrarily which bubbles up into throwing an exception erroneously early. For example we've historically returned -1 (granted, representing millis so it might "just work" here) from RpcThrottlingException#getWaitInterval in cases which cannot be parsed:

// Visible for TestRpcThrottlingException
protected static long timeFromString(String timeDiff) {
Pattern pattern =
Pattern.compile("^(?:(\\d+)hrs?, )?(?:(\\d+)mins?, )?(?:(\\d+)sec[, ]{0,2})?(?:(\\d+)ms)?");
long[] factors = new long[] { 60 * 60 * 1000, 60 * 1000, 1000, 1 };
Matcher m = pattern.matcher(timeDiff);
if (m.find()) {
int numGroups = m.groupCount();
long time = 0;
for (int j = 1; j <= numGroups; j++) {
String group = m.group(j);
if (group == null) {
continue;
}
time += Math.round(Float.parseFloat(group) * factors[j - 1]);
}
return time;
}
return -1;
}

long expectedSleepNs;
if (error instanceof RpcThrottlingException) {
RpcThrottlingException rpcThrottlingException = (RpcThrottlingException) error;
expectedSleepNs = TimeUnit.MILLISECONDS.toNanos(rpcThrottlingException.getWaitInterval());
if (expectedSleepNs > remainingTimeNs) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Better add a log here to mention that we will give up retrying since the remaining time is not enough for the next retry because of the server is throttling us.

if (LOG.isDebugEnabled()) {
LOG.debug("RpcThrottlingException suggested pause of {}ns which would exceed "
+ "the timeout. We should throw instead.", expectedSleepNs, rpcThrottlingException);
}
return OptionalLong.empty();
}
if (LOG.isDebugEnabled()) {
LOG.debug("Sleeping for {}ns after catching RpcThrottlingException", expectedSleepNs,
rpcThrottlingException);
}
} else {
expectedSleepNs =
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;
}
return OptionalLong.of(expectedSleepNs);
}

}
Loading