You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bb...@apache.org on 2022/11/15 20:41:19 UTC
[hbase] branch branch-2 updated: HBASE-26809: Report client backoff time for server overloaded (#4786)
This is an automated email from the ASF dual-hosted git repository.
bbeaudreault pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2 by this push:
new 53e872e5f18 HBASE-26809: Report client backoff time for server overloaded (#4786)
53e872e5f18 is described below
commit 53e872e5f185d7900e30d16727938acd7456149b
Author: Bri Augenreich <bb...@vt.edu>
AuthorDate: Tue Nov 15 15:41:08 2022 -0500
HBASE-26809: Report client backoff time for server overloaded (#4786)
Co-authored-by: Briana Augenreich <ba...@hubspot.com>
Signed-off-by: Bryan Beaudreault <bb...@apache.org>
---
.../hbase/client/AsyncBatchRpcRetryingCaller.java | 4 ++++
.../hbase/client/AsyncRequestFutureImpl.java | 6 +++++
.../hbase/client/AsyncRpcRetryingCaller.java | 5 ++++
.../AsyncScanSingleRegionRpcRetryingCaller.java | 7 ++++++
.../hbase/client/ConnectionImplementation.java | 17 ++++++++++----
.../org/apache/hadoop/hbase/client/HTable.java | 5 ++--
.../hadoop/hbase/client/HTableMultiplexer.java | 3 ++-
.../hadoop/hbase/client/MetricsConnection.java | 8 +++++++
.../hbase/client/RpcRetryingCallerFactory.java | 27 ++++++++++++++--------
.../hadoop/hbase/client/RpcRetryingCallerImpl.java | 15 ++++++------
.../hbase/client/ScannerCallableWithReplicas.java | 12 ++++++----
.../hadoop/hbase/client/SecureBulkLoadClient.java | 4 ++--
.../hadoop/hbase/client/TestAsyncProcess.java | 8 ++++---
.../TestAsyncProcessWithRegionException.java | 3 ++-
.../hbase/client/TestRpcRetryingCallerImpl.java | 4 ++--
.../hadoop/hbase/regionserver/HRegionServer.java | 3 ++-
.../RegionReplicaReplicationEndpoint.java | 4 ++--
.../hadoop/hbase/tool/LoadIncrementalHFiles.java | 2 +-
.../hbase/client/HConnectionTestingUtility.java | 5 ++--
...stRegionReplicaReplicationEndpointNoMaster.java | 4 ++--
20 files changed, 101 insertions(+), 45 deletions(-)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
index 0798915c08d..7bee885586b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
@@ -487,6 +487,10 @@ class AsyncBatchRpcRetryingCaller<T> {
} else {
delayNs = getPauseTime(pauseNsToUse, tries - 1);
}
+ if (isServerOverloaded) {
+ Optional<MetricsConnection> metrics = conn.getConnectionMetrics();
+ metrics.ifPresent(m -> m.incrementServerOverloadedBackoffTime(delayNs, TimeUnit.NANOSECONDS));
+ }
retryTimer.newTimeout(t -> groupAndSend(actions, tries + 1), delayNs, TimeUnit.NANOSECONDS);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
index 987d86bd8aa..a91fd5af6af 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
@@ -747,6 +747,12 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
backOffTime = errorsByServer.calculateBackoffTime(oldServer,
asyncProcess.connectionConfiguration.getPauseMillis());
}
+
+ MetricsConnection metrics = asyncProcess.connection.getConnectionMetrics();
+ if (metrics != null && HBaseServerException.isServerOverloaded(throwable)) {
+ metrics.incrementServerOverloadedBackoffTime(backOffTime, TimeUnit.MILLISECONDS);
+ }
+
if (numAttempt > asyncProcess.startLogErrorsCnt) {
// We use this value to have some logs when we have multiple failures, but not too many
// logs, as errors are to be expected when a region moves, splits and so on
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
index 95ed97e1811..d0bbe4b5fa3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
@@ -139,6 +139,11 @@ public abstract class AsyncRpcRetryingCaller<T> {
delayNs = getPauseTime(pauseNsToUse, tries - 1);
}
tries++;
+
+ if (HBaseServerException.isServerOverloaded(error)) {
+ Optional<MetricsConnection> metrics = conn.getConnectionMetrics();
+ metrics.ifPresent(m -> m.incrementServerOverloadedBackoffTime(delayNs, TimeUnit.NANOSECONDS));
+ }
retryTimer.newTimeout(t -> doCall(), delayNs, TimeUnit.NANOSECONDS);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
index dbaae5c26e2..2653b3c75b3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
@@ -113,6 +113,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
private final Runnable completeWhenNoMoreResultsInRegion;
+ protected final AsyncConnectionImpl conn;
+
private final CompletableFuture<Boolean> future;
private final HBaseRpcController controller;
@@ -318,6 +320,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
long pauseNsForServerOverloaded, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs,
int startLogErrorsCnt) {
this.retryTimer = retryTimer;
+ this.conn = conn;
this.scan = scan;
this.scanMetrics = scanMetrics;
this.scannerId = scannerId;
@@ -441,6 +444,10 @@ class AsyncScanSingleRegionRpcRetryingCaller {
return;
}
tries++;
+ if (HBaseServerException.isServerOverloaded(error)) {
+ Optional<MetricsConnection> metrics = conn.getConnectionMetrics();
+ metrics.ifPresent(m -> m.incrementServerOverloadedBackoffTime(delayNs, TimeUnit.NANOSECONDS));
+ }
retryTimer.newTimeout(t -> call(), delayNs, TimeUnit.NANOSECONDS);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 4d231cfe0b9..a834da72d28 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -301,10 +301,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
this.stats = ServerStatisticTracker.create(conf);
this.interceptor = new RetryingCallerInterceptorFactory(conf).build();
- this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
- this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats);
+
this.backoffPolicy = ClientBackoffPolicyFactory.create(conf);
- this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory);
boolean shouldListen =
conf.getBoolean(HConstants.STATUS_PUBLISHED, HConstants.STATUS_PUBLISHED_DEFAULT);
@@ -332,6 +330,10 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
this.metaCache = new MetaCache(this.metrics);
this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId, this.metrics);
+ this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+ this.rpcCallerFactory =
+ RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats, this.metrics);
+ this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory);
// Do we publish the status?
if (shouldListen) {
@@ -1058,6 +1060,11 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
// Only relocate the parent region if necessary
relocateMeta =
!(e instanceof RegionOfflineException || e instanceof NoServerForRegionException);
+
+ if (metrics != null && HBaseServerException.isServerOverloaded(e)) {
+ metrics.incrementServerOverloadedBackoffTime(
+ ConnectionUtils.getPauseTime(pauseBase, tries), TimeUnit.MILLISECONDS);
+ }
} finally {
userRegionLock.unlock();
}
@@ -2191,8 +2198,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
@Override
public RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf) {
- return RpcRetryingCallerFactory.instantiate(conf, this.interceptor,
- this.getStatisticsTracker());
+ return RpcRetryingCallerFactory.instantiate(conf, this.interceptor, this.getStatisticsTracker(),
+ metrics);
}
@Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index f36c4af0de0..80325abd7f9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -1305,8 +1305,9 @@ public class HTable implements Table {
final List<String> callbackErrorServers = new ArrayList<>();
Object[] results = new Object[execs.size()];
- AsyncProcess asyncProcess = new AsyncProcess(connection, configuration,
- RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()),
+ AsyncProcess asyncProcess = new AsyncProcess(
+ connection, configuration, RpcRetryingCallerFactory.instantiate(configuration,
+ connection.getStatisticsTracker(), connection.getConnectionMetrics()),
RpcControllerFactory.instantiate(configuration));
Batch.Callback<ClientProtos.CoprocessorServiceResult> resultsCallback =
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index c4c95d73c2b..6b54149f9f9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -423,7 +423,8 @@ public class HTableMultiplexer {
this.addr = addr;
this.multiplexer = htableMultiplexer;
this.queue = new LinkedBlockingQueue<>(perRegionServerBufferQueueSize);
- RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf);
+ RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf,
+ conn == null ? null : conn.getConnectionMetrics());
RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf);
this.writeRpcTimeout = conf.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
index dc452bcd9d9..2ad2944cddd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
@@ -315,6 +315,7 @@ public class MetricsConnection implements StatisticTrackable {
protected final Histogram numActionsPerServerHist;
protected final Counter nsLookups;
protected final Counter nsLookupsFailed;
+ protected final Timer overloadedBackoffTimer;
// dynamic metrics
@@ -377,6 +378,9 @@ public class MetricsConnection implements StatisticTrackable {
this.nsLookups = registry.counter(name(this.getClass(), NS_LOOKUPS, scope));
this.nsLookupsFailed = registry.counter(name(this.getClass(), NS_LOOKUPS_FAILED, scope));
+ this.overloadedBackoffTimer =
+ registry.timer(name(this.getClass(), "overloadedBackoffDurationMs", scope));
+
this.reporter = JmxReporter.forRegistry(this.registry).build();
this.reporter.start();
}
@@ -449,6 +453,10 @@ public class MetricsConnection implements StatisticTrackable {
this.runnerStats.updateDelayInterval(interval);
}
+ public void incrementServerOverloadedBackoffTime(long time, TimeUnit timeUnit) {
+ overloadedBackoffTimer.update(time, timeUnit);
+ }
+
/**
* Get a metric for {@code key} from {@code map}, or create it with {@code factory}.
*/
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
index a1991378228..3e8545f6a38 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
@@ -33,17 +33,20 @@ public class RpcRetryingCallerFactory {
private final ConnectionConfiguration connectionConf;
private final RetryingCallerInterceptor interceptor;
private final int startLogErrorsCnt;
+ private final MetricsConnection metrics;
public RpcRetryingCallerFactory(Configuration conf) {
- this(conf, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR);
+ this(conf, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null);
}
- public RpcRetryingCallerFactory(Configuration conf, RetryingCallerInterceptor interceptor) {
+ public RpcRetryingCallerFactory(Configuration conf, RetryingCallerInterceptor interceptor,
+ MetricsConnection metrics) {
this.conf = conf;
this.connectionConf = new ConnectionConfiguration(conf);
startLogErrorsCnt = conf.getInt(AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY,
AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
this.interceptor = interceptor;
+ this.metrics = metrics;
}
/**
@@ -54,7 +57,7 @@ public class RpcRetryingCallerFactory {
// is cheap as it does not require parsing a complex structure.
return new RpcRetryingCallerImpl<>(connectionConf.getPauseMillis(),
connectionConf.getPauseMillisForServerOverloaded(), connectionConf.getRetriesNumber(),
- interceptor, startLogErrorsCnt, rpcTimeout);
+ interceptor, startLogErrorsCnt, rpcTimeout, metrics);
}
/**
@@ -65,26 +68,30 @@ public class RpcRetryingCallerFactory {
// is cheap as it does not require parsing a complex structure.
return new RpcRetryingCallerImpl<>(connectionConf.getPauseMillis(),
connectionConf.getPauseMillisForServerOverloaded(), connectionConf.getRetriesNumber(),
- interceptor, startLogErrorsCnt, connectionConf.getRpcTimeout());
+ interceptor, startLogErrorsCnt, connectionConf.getRpcTimeout(), metrics);
}
- public static RpcRetryingCallerFactory instantiate(Configuration configuration) {
- return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null);
+ public static RpcRetryingCallerFactory instantiate(Configuration configuration,
+ MetricsConnection metrics) {
+ return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null,
+ metrics);
}
public static RpcRetryingCallerFactory instantiate(Configuration configuration,
- ServerStatisticTracker stats) {
- return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, stats);
+ ServerStatisticTracker stats, MetricsConnection metrics) {
+ return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, stats,
+ metrics);
}
public static RpcRetryingCallerFactory instantiate(Configuration configuration,
- RetryingCallerInterceptor interceptor, ServerStatisticTracker stats) {
+ RetryingCallerInterceptor interceptor, ServerStatisticTracker stats,
+ MetricsConnection metrics) {
String clazzName = RpcRetryingCallerFactory.class.getName();
String rpcCallerFactoryClazz =
configuration.get(RpcRetryingCallerFactory.CUSTOM_CALLER_CONF_KEY, clazzName);
RpcRetryingCallerFactory factory;
if (rpcCallerFactoryClazz.equals(clazzName)) {
- factory = new RpcRetryingCallerFactory(configuration, interceptor);
+ factory = new RpcRetryingCallerFactory(configuration, interceptor, metrics);
} else {
factory = ReflectionUtils.instantiateWithCustomCtor(rpcCallerFactoryClazz,
new Class[] { Configuration.class }, new Object[] { configuration });
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
index 67b12b8a34b..4d88e34ff65 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
@@ -26,6 +26,7 @@ import java.net.SocketTimeoutException;
import java.time.Instant;
import java.util.ArrayList;
import java.util.List;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseServerException;
@@ -63,15 +64,11 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
private final RetryingCallerInterceptor interceptor;
private final RetryingCallerInterceptorContext context;
private final RetryingTimeTracker tracker;
+ private final MetricsConnection metrics;
public RpcRetryingCallerImpl(long pause, long pauseForServerOverloaded, int retries,
- int startLogErrorsCnt) {
- this(pause, pauseForServerOverloaded, retries,
- RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, startLogErrorsCnt, 0);
- }
-
- public RpcRetryingCallerImpl(long pause, long pauseForServerOverloaded, int retries,
- RetryingCallerInterceptor interceptor, int startLogErrorsCnt, int rpcTimeout) {
+ RetryingCallerInterceptor interceptor, int startLogErrorsCnt, int rpcTimeout,
+ MetricsConnection metricsConnection) {
this.pause = pause;
this.pauseForServerOverloaded = pauseForServerOverloaded;
this.maxAttempts = retries2Attempts(retries);
@@ -80,6 +77,7 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
this.startLogErrorsCnt = startLogErrorsCnt;
this.tracker = new RetryingTimeTracker();
this.rpcTimeout = rpcTimeout;
+ this.metrics = metricsConnection;
}
@Override
@@ -158,6 +156,9 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
+ t.getMessage() + " " + callable.getExceptionMessageAdditionalDetail();
throw (SocketTimeoutException) new SocketTimeoutException(msg).initCause(t);
}
+ if (metrics != null && HBaseServerException.isServerOverloaded(t)) {
+ metrics.incrementServerOverloadedBackoffTime(expectedSleep, TimeUnit.MILLISECONDS);
+ }
} finally {
interceptor.updateFailureInfo(context);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index fe155136cb2..27cc4d15126 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -179,8 +179,9 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
// We want to accomodate some RPCs for redundant replica scans (but are still in progress)
ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs =
new ResultBoundedCompletionService<>(
- RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf), pool,
- regionReplication * 5);
+ RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf,
+ cConnection == null ? null : cConnection.getConnectionMetrics()),
+ pool, regionReplication * 5);
AtomicBoolean done = new AtomicBoolean(false);
replicaSwitched.set(false);
@@ -381,8 +382,11 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
// and we can't invoke it multiple times at the same time)
this.caller = ScannerCallableWithReplicas.this.caller;
if (scan.getConsistency() == Consistency.TIMELINE) {
- this.caller = RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf).<
- Result[]> newCaller();
+ this.caller =
+ RpcRetryingCallerFactory
+ .instantiate(ScannerCallableWithReplicas.this.conf,
+ cConnection == null ? null : cConnection.getConnectionMetrics())
+ .<Result[]> newCaller();
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
index 825a58e7bdd..1838d78eb9f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
@@ -69,7 +69,7 @@ public class SecureBulkLoadClient {
return response.getBulkToken();
}
};
- return RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
+ return RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null, null)
.<String> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
} catch (Throwable throwable) {
throw new IOException(throwable);
@@ -91,7 +91,7 @@ public class SecureBulkLoadClient {
return null;
}
};
- RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null).<Void> newCaller()
+ RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null, null).<Void> newCaller()
.callWithRetries(callable, Integer.MAX_VALUE);
} catch (Throwable throwable) {
throw new IOException(throwable);
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index de6d6012322..a55001e1627 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -246,7 +246,8 @@ public class TestAsyncProcess {
}
});
- return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 10, 9) {
+ return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 10,
+ RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, 9, 0, null) {
@Override
public AbstractResponse callWithoutRetries(RetryingCallable<AbstractResponse> callable,
int callTimeout) throws IOException, RuntimeException {
@@ -307,7 +308,7 @@ public class TestAsyncProcess {
private final IOException e;
public CallerWithFailure(IOException e) {
- super(100, 500, 100, 9);
+ super(100, 500, 100, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, 9, 0, null);
this.e = e;
}
@@ -412,7 +413,8 @@ public class TestAsyncProcess {
replicaCalls.incrementAndGet();
}
- return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 10, 9) {
+ return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 10,
+ RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, 9, 0, null) {
@Override
public MultiResponse callWithoutRetries(RetryingCallable<AbstractResponse> callable,
int callTimeout) throws IOException, RuntimeException {
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
index ca9aabfef04..98c13761262 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
@@ -222,7 +222,8 @@ public class TestAsyncProcessWithRegionException {
});
});
mr.addException(REGION_INFO.getRegionName(), IOE);
- return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 0, 9) {
+ return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 0,
+ RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, 9, 0, null) {
@Override
public AbstractResponse callWithoutRetries(RetryingCallable<AbstractResponse> callable,
int callTimeout) {
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcRetryingCallerImpl.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcRetryingCallerImpl.java
index 3d3d64f4c21..a0177190035 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcRetryingCallerImpl.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcRetryingCallerImpl.java
@@ -58,8 +58,8 @@ public class TestRpcRetryingCallerImpl {
long pauseMillis = 1;
long specialPauseMillis = 2;
- RpcRetryingCallerImpl<Void> caller =
- new RpcRetryingCallerImpl<>(pauseMillis, specialPauseMillis, 2, 0);
+ RpcRetryingCallerImpl<Void> caller = new RpcRetryingCallerImpl<>(pauseMillis,
+ specialPauseMillis, 2, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, 0, 0, null);
RetryingCallable<Void> callable =
new ThrowingCallable(CallQueueTooBigException.class, specialPauseMillis);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index c3589d57df1..fe634dec5b5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -667,7 +667,8 @@ public class HRegionServer extends Thread
serverName = ServerName.valueOf(hostName, this.rpcServices.isa.getPort(), this.startcode);
rpcControllerFactory = RpcControllerFactory.instantiate(this.conf);
- rpcRetryingCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
+ rpcRetryingCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf,
+ clusterConnection == null ? null : clusterConnection.getConnectionMetrics());
// login the zookeeper client principal (if using security)
ZKAuthentication.loginClient(this.conf, HConstants.ZK_CLIENT_KEYTAB_FILE,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index 21a2d51326a..cf0f69372d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -390,8 +390,8 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
this.sink = sink;
this.connection = connection;
this.operationTimeout = operationTimeout;
- this.rpcRetryingCallerFactory =
- RpcRetryingCallerFactory.instantiate(connection.getConfiguration());
+ this.rpcRetryingCallerFactory = RpcRetryingCallerFactory
+ .instantiate(connection.getConfiguration(), connection.getConnectionMetrics());
this.rpcControllerFactory = RpcControllerFactory.instantiate(connection.getConfiguration());
this.pool = pool;
this.tableDescriptors = tableDescriptors;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
index 5e3f2e9468f..54adfd22a36 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
@@ -871,7 +871,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
List<LoadQueueItem> toRetry = new ArrayList<>();
try {
Configuration conf = getConf();
- byte[] region = RpcRetryingCallerFactory.instantiate(conf, null).<byte[]> newCaller()
+ byte[] region = RpcRetryingCallerFactory.instantiate(conf, null, null).<byte[]> newCaller()
.callWithRetries(serviceCallable, Integer.MAX_VALUE);
if (region == null) {
LOG.warn("Attempt to bulk load region containing " + Bytes.toStringBinary(first)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index de4e7d41ed6..139d8bf8b66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -123,10 +123,11 @@ public class HConnectionTestingUtility {
NonceGenerator ng = Mockito.mock(NonceGenerator.class);
Mockito.when(c.getNonceGenerator()).thenReturn(ng);
AsyncProcess asyncProcess = new AsyncProcess(c, conf,
- RpcRetryingCallerFactory.instantiate(conf), RpcControllerFactory.instantiate(conf));
+ RpcRetryingCallerFactory.instantiate(conf, c.getConnectionMetrics()),
+ RpcControllerFactory.instantiate(conf));
Mockito.when(c.getAsyncProcess()).thenReturn(asyncProcess);
Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn(RpcRetryingCallerFactory
- .instantiate(conf, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null));
+ .instantiate(conf, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null, null));
Mockito.when(c.getRpcControllerFactory()).thenReturn(Mockito.mock(RpcControllerFactory.class));
Table t = Mockito.mock(Table.class);
Mockito.when(c.getTable((TableName) Mockito.any())).thenReturn(t);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index da95c6fa7d8..7d588712c33 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -209,8 +209,8 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
locations.getRegionLocation(1), locations.getRegionLocation(1).getRegionInfo(), row,
Lists.newArrayList(entry), new AtomicLong());
- RpcRetryingCallerFactory factory =
- RpcRetryingCallerFactory.instantiate(connection.getConfiguration());
+ RpcRetryingCallerFactory factory = RpcRetryingCallerFactory
+ .instantiate(connection.getConfiguration(), connection.getConnectionMetrics());
factory.<ReplicateWALEntryResponse> newCaller().callWithRetries(callable, 10000);
}
}