-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-27798: Client side should back off based on wait interval in RpcThrottlingException #5275
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
13c6be0
aa0c3f1
761efd0
7027912
fc8eb20
902f8db
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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; | ||
|
|
@@ -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; | ||
|
|
@@ -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; | ||
|
|
@@ -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 { | ||
|
|
@@ -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; | ||
|
|
@@ -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); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -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) { | ||
|
|
@@ -529,7 +535,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); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| 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) { | ||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||
| // 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; | |
| } |
Outdated
There was a problem hiding this comment.
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.
bbeaudreault marked this conversation as resolved.
Show resolved
Hide resolved
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,86 @@ | ||
| /* | ||
| * 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 static org.junit.Assert.assertEquals; | ||
| import static org.junit.Assert.assertFalse; | ||
| import static org.junit.Assert.assertTrue; | ||
|
|
||
| import java.util.OptionalLong; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.apache.hadoop.hbase.HBaseClassTestRule; | ||
| import org.apache.hadoop.hbase.HBaseServerException; | ||
| import org.apache.hadoop.hbase.quotas.RpcThrottlingException; | ||
| import org.apache.hadoop.hbase.testclassification.ClientTests; | ||
| import org.apache.hadoop.hbase.testclassification.SmallTests; | ||
| import org.junit.ClassRule; | ||
| import org.junit.Test; | ||
| import org.junit.experimental.categories.Category; | ||
|
|
||
| @Category({ ClientTests.class, SmallTests.class }) | ||
| public class TestHBaseServerExceptionPauseManager { | ||
|
|
||
| private static final long WAIT_INTERVAL_MILLIS = 1L; | ||
| private static final long WAIT_INTERVAL_NANOS = | ||
| TimeUnit.MILLISECONDS.toNanos(WAIT_INTERVAL_MILLIS); | ||
| private static final long PAUSE_NANOS_FOR_SERVER_OVERLOADED = WAIT_INTERVAL_NANOS * 3; | ||
|
|
||
| private static final long PAUSE_NANOS = WAIT_INTERVAL_NANOS * 2; | ||
|
|
||
| private final RpcThrottlingException RPC_THROTTLING_EXCEPTION = new RpcThrottlingException( | ||
| RpcThrottlingException.Type.NumRequestsExceeded, WAIT_INTERVAL_MILLIS, "doot"); | ||
| private final Throwable OTHER_EXCEPTION = new RuntimeException(""); | ||
| private final HBaseServerException SERVER_OVERLOADED_EXCEPTION = new HBaseServerException(true); | ||
|
|
||
| private final HBaseServerExceptionPauseManager pauseManager = | ||
| new HBaseServerExceptionPauseManager(PAUSE_NANOS, PAUSE_NANOS_FOR_SERVER_OVERLOADED); | ||
|
|
||
| @ClassRule | ||
| public static final HBaseClassTestRule CLASS_RULE = | ||
| HBaseClassTestRule.forClass(TestHBaseServerExceptionPauseManager.class); | ||
|
|
||
| @Test | ||
| public void itSupportsRpcThrottlingNanos() { | ||
| OptionalLong pauseNanos = | ||
| pauseManager.getPauseNsFromException(RPC_THROTTLING_EXCEPTION, Long.MAX_VALUE); | ||
| assertTrue(pauseNanos.isPresent()); | ||
| assertEquals(pauseNanos.getAsLong(), WAIT_INTERVAL_NANOS); | ||
| } | ||
|
|
||
| @Test | ||
| public void itSupportsServerOverloadedExceptionNanos() { | ||
| OptionalLong pauseNanos = | ||
| pauseManager.getPauseNsFromException(SERVER_OVERLOADED_EXCEPTION, Long.MAX_VALUE); | ||
| assertTrue(pauseNanos.isPresent()); | ||
| assertEquals(pauseNanos.getAsLong(), PAUSE_NANOS_FOR_SERVER_OVERLOADED); | ||
| } | ||
|
|
||
| @Test | ||
| public void itSupportsOtherExceptionNanos() { | ||
| OptionalLong pauseNanos = pauseManager.getPauseNsFromException(OTHER_EXCEPTION, Long.MAX_VALUE); | ||
| assertTrue(pauseNanos.isPresent()); | ||
| assertEquals(pauseNanos.getAsLong(), PAUSE_NANOS); | ||
| } | ||
|
|
||
| @Test | ||
| public void itThrottledTimeoutFastFail() { | ||
| OptionalLong pauseNanos = pauseManager.getPauseNsFromException(RPC_THROTTLING_EXCEPTION, 0L); | ||
| assertFalse(pauseNanos.isPresent()); | ||
| } | ||
|
|
||
| } |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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