Skip to content
Merged
Show file tree
Hide file tree
Changes from 7 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 @@ -360,7 +362,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 +467,26 @@ 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 = HBaseServerExceptionPauseManager.getPauseNsFromException(error,
pauseNs, pauseNsForServerOverloaded, 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 +538,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 Down Expand Up @@ -125,8 +127,14 @@ protected final void resetCallTimeout() {
}

private void tryScheduleRetry(Throwable error) {
long pauseNsToUse =
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;
OptionalLong maybePauseNsToUse = HBaseServerExceptionPauseManager.getPauseNsFromException(error,
pauseNs, pauseNsForServerOverloaded, 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 @@ -419,8 +421,15 @@ private void onError(Throwable error) {
return;
}
long delayNs;
long pauseNsToUse =
HBaseServerException.isServerOverloaded(error) ? pauseNsForServerOverloaded : pauseNs;

OptionalLong maybePauseNsToUse = HBaseServerExceptionPauseManager.getPauseNsFromException(error,
pauseNs, pauseNsForServerOverloaded, 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,54 @@
/*
* 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 HBaseServerExceptionPauseManager() {}
Copy link
Contributor

Choose a reason for hiding this comment

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

You could pass the Optional in here. You could also pass in the pauseNs values if you want, since they don't change. But not a huge deal

Copy link
Contributor Author

Choose a reason for hiding this comment

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

which optional are you referring to? I'm envisioning something like:

  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);
  }

Copy link
Contributor

Choose a reason for hiding this comment

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

In one of my other most recent comments I suggested doing the metrics incrementing for server overloaded in here so we don't have to handle server overloaded in two places. Not sure if that works, but if so the metrics instance could get passed in 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.

Oh yeah, I see what you mean! That would be nice. I'm going to omit it here because it would require a somewhat larger diff; the metric reported is the delayNs rather than just the pauseNs, there's some nuance in whether the metric gets reported in the fast failure cases, etc. so I think it would complicate the changeset


public static OptionalLong getPauseNsFromException(Throwable error, long pauseNs,
long pauseNsForServerOverloaded, 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);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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);

@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestHBaseServerExceptionPauseManager.class);

@Test
public void itSupportsRpcThrottlingNanos() {
OptionalLong pauseNanos = HBaseServerExceptionPauseManager.getPauseNsFromException(
RPC_THROTTLING_EXCEPTION, PAUSE_NANOS, PAUSE_NANOS_FOR_SERVER_OVERLOADED, Long.MAX_VALUE);
assertTrue(pauseNanos.isPresent());
assertEquals(pauseNanos.getAsLong(), WAIT_INTERVAL_NANOS);
}

@Test
public void itSupportsServerOverloadedExceptionNanos() {
OptionalLong pauseNanos = HBaseServerExceptionPauseManager.getPauseNsFromException(
SERVER_OVERLOADED_EXCEPTION, PAUSE_NANOS, PAUSE_NANOS_FOR_SERVER_OVERLOADED, Long.MAX_VALUE);
assertTrue(pauseNanos.isPresent());
assertEquals(pauseNanos.getAsLong(), PAUSE_NANOS_FOR_SERVER_OVERLOADED);
}

@Test
public void itSupportsOtherExceptionNanos() {
OptionalLong pauseNanos = HBaseServerExceptionPauseManager.getPauseNsFromException(
OTHER_EXCEPTION, PAUSE_NANOS, PAUSE_NANOS_FOR_SERVER_OVERLOADED, Long.MAX_VALUE);
assertTrue(pauseNanos.isPresent());
assertEquals(pauseNanos.getAsLong(), PAUSE_NANOS);
}

@Test
public void itThrottledTimeoutFastFail() {
OptionalLong pauseNanos = HBaseServerExceptionPauseManager.getPauseNsFromException(
RPC_THROTTLING_EXCEPTION, PAUSE_NANOS, PAUSE_NANOS_FOR_SERVER_OVERLOADED, 0L);
assertFalse(pauseNanos.isPresent());
}

}
Loading