Skip to content

Commit 0add475

Browse files
rmdmattinglyRay Mattingly
authored andcommitted
HBASE-27798 Client side should back off based on wait interval in RpcThrottlingException (apache#5275)
Signed-off-by: Bryan Beaudreault <bbeaudreault@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
1 parent 0121eeb commit 0add475

7 files changed

Lines changed: 655 additions & 77 deletions

File tree

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java

Lines changed: 18 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,7 @@
1818
package org.apache.hadoop.hbase.client;
1919

2020
import static org.apache.hadoop.hbase.CellUtil.createCellScanner;
21-
import static org.apache.hadoop.hbase.client.ConnectionUtils.SLEEP_DELTA_NS;
2221
import static org.apache.hadoop.hbase.client.ConnectionUtils.calcPriority;
23-
import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
2422
import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
2523
import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
2624
import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
@@ -35,6 +33,7 @@
3533
import java.util.List;
3634
import java.util.Map;
3735
import java.util.Optional;
36+
import java.util.OptionalLong;
3837
import java.util.concurrent.CompletableFuture;
3938
import java.util.concurrent.ConcurrentHashMap;
4039
import java.util.concurrent.ConcurrentLinkedQueue;
@@ -56,6 +55,7 @@
5655
import org.apache.hadoop.hbase.client.MultiResponse.RegionResult;
5756
import org.apache.hadoop.hbase.client.RetriesExhaustedException.ThrowableWithExtraContext;
5857
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
58+
import org.apache.hadoop.hbase.client.backoff.HBaseServerExceptionPauseManager;
5959
import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
6060
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
6161
import org.apache.hadoop.hbase.util.Bytes;
@@ -102,10 +102,6 @@ class AsyncBatchRpcRetryingCaller<T> {
102102

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

105-
private final long pauseNs;
106-
107-
private final long pauseNsForServerOverloaded;
108-
109105
private final int maxAttempts;
110106

111107
private final long operationTimeoutNs;
@@ -116,6 +112,8 @@ class AsyncBatchRpcRetryingCaller<T> {
116112

117113
private final long startNs;
118114

115+
private final HBaseServerExceptionPauseManager pauseManager;
116+
119117
// we can not use HRegionLocation as the map key because the hashCode and equals method of
120118
// HRegionLocation only consider serverName.
121119
private static final class RegionRequest {
@@ -155,8 +153,6 @@ public AsyncBatchRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn,
155153
this.retryTimer = retryTimer;
156154
this.conn = conn;
157155
this.tableName = tableName;
158-
this.pauseNs = pauseNs;
159-
this.pauseNsForServerOverloaded = pauseNsForServerOverloaded;
160156
this.maxAttempts = maxAttempts;
161157
this.operationTimeoutNs = operationTimeoutNs;
162158
this.rpcTimeoutNs = rpcTimeoutNs;
@@ -182,6 +178,8 @@ public AsyncBatchRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn,
182178
}
183179
this.action2Errors = new IdentityHashMap<>();
184180
this.startNs = System.nanoTime();
181+
this.pauseManager =
182+
new HBaseServerExceptionPauseManager(pauseNs, pauseNsForServerOverloaded, operationTimeoutNs);
185183
}
186184

187185
private static boolean hasIncrementOrAppend(Row action) {
@@ -204,10 +202,6 @@ private static boolean hasIncrementOrAppend(RowMutations mutations) {
204202
return false;
205203
}
206204

207-
private long remainingTimeNs() {
208-
return operationTimeoutNs - (System.nanoTime() - startNs);
209-
}
210-
211205
private List<ThrowableWithExtraContext> removeErrors(Action action) {
212206
synchronized (action2Errors) {
213207
return action2Errors.remove(action);
@@ -360,14 +354,14 @@ private void onComplete(Map<byte[], RegionRequest> actionsByRegion, int tries,
360354
}
361355
});
362356
if (!failedActions.isEmpty()) {
363-
tryResubmit(failedActions.stream(), tries, retryImmediately.booleanValue(), false);
357+
tryResubmit(failedActions.stream(), tries, retryImmediately.booleanValue(), null);
364358
}
365359
}
366360

367361
private void sendToServer(ServerName serverName, ServerRequest serverReq, int tries) {
368362
long remainingNs;
369363
if (operationTimeoutNs > 0) {
370-
remainingNs = remainingTimeNs();
364+
remainingNs = pauseManager.remainingTimeNs(startNs);
371365
if (remainingNs <= 0) {
372366
failAll(serverReq.actionsByRegion.values().stream().flatMap(r -> r.actions.stream()),
373367
tries);
@@ -465,29 +459,22 @@ private void onError(Map<byte[], RegionRequest> actionsByRegion, int tries, Thro
465459
List<Action> copiedActions = actionsByRegion.values().stream().flatMap(r -> r.actions.stream())
466460
.collect(Collectors.toList());
467461
addError(copiedActions, error, serverName);
468-
tryResubmit(copiedActions.stream(), tries, error instanceof RetryImmediatelyException,
469-
HBaseServerException.isServerOverloaded(error));
462+
tryResubmit(copiedActions.stream(), tries, error instanceof RetryImmediatelyException, error);
470463
}
471464

472465
private void tryResubmit(Stream<Action> actions, int tries, boolean immediately,
473-
boolean isServerOverloaded) {
466+
Throwable error) {
474467
if (immediately) {
475468
groupAndSend(actions, tries);
476469
return;
477470
}
478-
long delayNs;
479-
long pauseNsToUse = isServerOverloaded ? pauseNsForServerOverloaded : pauseNs;
480-
if (operationTimeoutNs > 0) {
481-
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
482-
if (maxDelayNs <= 0) {
483-
failAll(actions, tries);
484-
return;
485-
}
486-
delayNs = Math.min(maxDelayNs, getPauseTime(pauseNsToUse, tries - 1));
487-
} else {
488-
delayNs = getPauseTime(pauseNsToUse, tries - 1);
471+
OptionalLong maybePauseNsToUse = pauseManager.getPauseNsFromException(error, tries, startNs);
472+
if (!maybePauseNsToUse.isPresent()) {
473+
failAll(actions, tries);
474+
return;
489475
}
490-
if (isServerOverloaded) {
476+
long delayNs = maybePauseNsToUse.getAsLong();
477+
if (HBaseServerException.isServerOverloaded(error)) {
491478
Optional<MetricsConnection> metrics = conn.getConnectionMetrics();
492479
metrics.ifPresent(m -> m.incrementServerOverloadedBackoffTime(delayNs, TimeUnit.NANOSECONDS));
493480
}
@@ -497,7 +484,7 @@ private void tryResubmit(Stream<Action> actions, int tries, boolean immediately,
497484
private void groupAndSend(Stream<Action> actions, int tries) {
498485
long locateTimeoutNs;
499486
if (operationTimeoutNs > 0) {
500-
locateTimeoutNs = remainingTimeNs();
487+
locateTimeoutNs = pauseManager.remainingTimeNs(startNs);
501488
if (locateTimeoutNs <= 0) {
502489
failAll(actions, tries);
503490
return;
@@ -528,7 +515,7 @@ private void groupAndSend(Stream<Action> actions, int tries) {
528515
sendOrDelay(actionsByServer, tries);
529516
}
530517
if (!locateFailed.isEmpty()) {
531-
tryResubmit(locateFailed.stream(), tries, false, false);
518+
tryResubmit(locateFailed.stream(), tries, false, null);
532519
}
533520
});
534521
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java

Lines changed: 12 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -17,14 +17,13 @@
1717
*/
1818
package org.apache.hadoop.hbase.client;
1919

20-
import static org.apache.hadoop.hbase.client.ConnectionUtils.SLEEP_DELTA_NS;
21-
import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
2220
import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
2321
import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
2422

2523
import java.util.ArrayList;
2624
import java.util.List;
2725
import java.util.Optional;
26+
import java.util.OptionalLong;
2827
import java.util.concurrent.CompletableFuture;
2928
import java.util.concurrent.TimeUnit;
3029
import java.util.function.Consumer;
@@ -35,6 +34,7 @@
3534
import org.apache.hadoop.hbase.TableName;
3635
import org.apache.hadoop.hbase.TableNotEnabledException;
3736
import org.apache.hadoop.hbase.TableNotFoundException;
37+
import org.apache.hadoop.hbase.client.backoff.HBaseServerExceptionPauseManager;
3838
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
3939
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
4040
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -56,10 +56,6 @@ public abstract class AsyncRpcRetryingCaller<T> {
5656

5757
private final long startNs;
5858

59-
private final long pauseNs;
60-
61-
private final long pauseNsForServerOverloaded;
62-
6359
private int tries = 1;
6460

6561
private final int maxAttempts;
@@ -78,14 +74,14 @@ public abstract class AsyncRpcRetryingCaller<T> {
7874

7975
protected final HBaseRpcController controller;
8076

77+
private final HBaseServerExceptionPauseManager pauseManager;
78+
8179
public AsyncRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, int priority,
8280
long pauseNs, long pauseNsForServerOverloaded, int maxAttempts, long operationTimeoutNs,
8381
long rpcTimeoutNs, int startLogErrorsCnt) {
8482
this.retryTimer = retryTimer;
8583
this.conn = conn;
8684
this.priority = priority;
87-
this.pauseNs = pauseNs;
88-
this.pauseNsForServerOverloaded = pauseNsForServerOverloaded;
8985
this.maxAttempts = maxAttempts;
9086
this.operationTimeoutNs = operationTimeoutNs;
9187
this.rpcTimeoutNs = rpcTimeoutNs;
@@ -95,14 +91,16 @@ public AsyncRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, int pr
9591
this.controller.setPriority(priority);
9692
this.exceptions = new ArrayList<>();
9793
this.startNs = System.nanoTime();
94+
this.pauseManager =
95+
new HBaseServerExceptionPauseManager(pauseNs, pauseNsForServerOverloaded, operationTimeoutNs);
9896
}
9997

10098
private long elapsedMs() {
10199
return TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNs);
102100
}
103101

104102
protected final long remainingTimeNs() {
105-
return operationTimeoutNs - (System.nanoTime() - startNs);
103+
return pauseManager.remainingTimeNs(startNs);
106104
}
107105

108106
protected final void completeExceptionally() {
@@ -125,19 +123,12 @@ protected final void resetCallTimeout() {
125123
}
126124

127125
private void tryScheduleRetry(Throwable error) {
128-
long pauseNsToUse =
129-
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;
130-
long delayNs;
131-
if (operationTimeoutNs > 0) {
132-
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
133-
if (maxDelayNs <= 0) {
134-
completeExceptionally();
135-
return;
136-
}
137-
delayNs = Math.min(maxDelayNs, getPauseTime(pauseNsToUse, tries - 1));
138-
} else {
139-
delayNs = getPauseTime(pauseNsToUse, tries - 1);
126+
OptionalLong maybePauseNsToUse = pauseManager.getPauseNsFromException(error, tries, startNs);
127+
if (!maybePauseNsToUse.isPresent()) {
128+
completeExceptionally();
129+
return;
140130
}
131+
long delayNs = maybePauseNsToUse.getAsLong();
141132
tries++;
142133

143134
if (HBaseServerException.isServerOverloaded(error)) {

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java

Lines changed: 13 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,6 @@
1717
*/
1818
package org.apache.hadoop.hbase.client;
1919

20-
import static org.apache.hadoop.hbase.client.ConnectionUtils.SLEEP_DELTA_NS;
21-
import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
2220
import static org.apache.hadoop.hbase.client.ConnectionUtils.incRPCCallsMetrics;
2321
import static org.apache.hadoop.hbase.client.ConnectionUtils.incRPCRetriesMetrics;
2422
import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan;
@@ -34,6 +32,7 @@
3432
import java.util.ArrayList;
3533
import java.util.List;
3634
import java.util.Optional;
35+
import java.util.OptionalLong;
3736
import java.util.concurrent.CompletableFuture;
3837
import java.util.concurrent.TimeUnit;
3938
import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -43,6 +42,7 @@
4342
import org.apache.hadoop.hbase.NotServingRegionException;
4443
import org.apache.hadoop.hbase.UnknownScannerException;
4544
import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer.ScanResumer;
45+
import org.apache.hadoop.hbase.client.backoff.HBaseServerExceptionPauseManager;
4646
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
4747
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
4848
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
@@ -99,10 +99,6 @@ class AsyncScanSingleRegionRpcRetryingCaller {
9999

100100
private final long scannerLeaseTimeoutPeriodNs;
101101

102-
private final long pauseNs;
103-
104-
private final long pauseNsForServerOverloaded;
105-
106102
private final int maxAttempts;
107103

108104
private final long scanTimeoutNs;
@@ -131,6 +127,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
131127

132128
private long nextCallSeq = -1L;
133129

130+
private final HBaseServerExceptionPauseManager pauseManager;
131+
134132
private enum ScanControllerState {
135133
INITIALIZED,
136134
SUSPENDED,
@@ -330,8 +328,6 @@ public AsyncScanSingleRegionRpcRetryingCaller(Timer retryTimer, AsyncConnectionI
330328
this.loc = loc;
331329
this.regionServerRemote = isRegionServerRemote;
332330
this.scannerLeaseTimeoutPeriodNs = scannerLeaseTimeoutPeriodNs;
333-
this.pauseNs = pauseNs;
334-
this.pauseNsForServerOverloaded = pauseNsForServerOverloaded;
335331
this.maxAttempts = maxAttempts;
336332
this.scanTimeoutNs = scanTimeoutNs;
337333
this.rpcTimeoutNs = rpcTimeoutNs;
@@ -346,16 +342,14 @@ public AsyncScanSingleRegionRpcRetryingCaller(Timer retryTimer, AsyncConnectionI
346342
this.controller = conn.rpcControllerFactory.newController();
347343
this.controller.setPriority(priority);
348344
this.exceptions = new ArrayList<>();
345+
this.pauseManager =
346+
new HBaseServerExceptionPauseManager(pauseNs, pauseNsForServerOverloaded, scanTimeoutNs);
349347
}
350348

351349
private long elapsedMs() {
352350
return TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - nextCallStartNs);
353351
}
354352

355-
private long remainingTimeNs() {
356-
return scanTimeoutNs - (System.nanoTime() - nextCallStartNs);
357-
}
358-
359353
private void closeScanner() {
360354
incRPCCallsMetrics(scanMetrics, regionServerRemote);
361355
resetController(controller, rpcTimeoutNs, HConstants.HIGH_QOS);
@@ -418,19 +412,14 @@ private void onError(Throwable error) {
418412
completeExceptionally(!scannerClosed);
419413
return;
420414
}
421-
long delayNs;
422-
long pauseNsToUse =
423-
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;
424-
if (scanTimeoutNs > 0) {
425-
long maxDelayNs = remainingTimeNs() - SLEEP_DELTA_NS;
426-
if (maxDelayNs <= 0) {
427-
completeExceptionally(!scannerClosed);
428-
return;
429-
}
430-
delayNs = Math.min(maxDelayNs, getPauseTime(pauseNsToUse, tries - 1));
431-
} else {
432-
delayNs = getPauseTime(pauseNsToUse, tries - 1);
415+
416+
OptionalLong maybePauseNsToUse =
417+
pauseManager.getPauseNsFromException(error, tries, nextCallStartNs);
418+
if (!maybePauseNsToUse.isPresent()) {
419+
completeExceptionally(!scannerClosed);
420+
return;
433421
}
422+
long delayNs = maybePauseNsToUse.getAsLong();
434423
if (scannerClosed) {
435424
completeWhenError(false);
436425
return;

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -280,7 +280,7 @@ static Result filterCells(Result result, Cell keepCellsAfter) {
280280
}
281281

282282
// Add a delta to avoid timeout immediately after a retry sleeping.
283-
static final long SLEEP_DELTA_NS = TimeUnit.MILLISECONDS.toNanos(1);
283+
public static final long SLEEP_DELTA_NS = TimeUnit.MILLISECONDS.toNanos(1);
284284

285285
static Get toCheckExistenceOnly(Get get) {
286286
if (get.isCheckExistenceOnly()) {

0 commit comments

Comments
 (0)