HBASE-26809: Report client backoff time for server overloaded (#4786)
Co-authored-by: Briana Augenreich <baugenreich@hubspot.com> Signed-off-by: Bryan Beaudreault <bbeaudreault@apache.org>
This commit is contained in:
parent
3880884e8e
commit
2d038edf4c
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -291,10 +291,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);
|
||||
|
@ -322,6 +320,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) {
|
||||
|
@ -1048,6 +1050,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();
|
||||
}
|
||||
|
@ -2175,8 +2182,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
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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}.
|
||||
*/
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
public static RpcRetryingCallerFactory instantiate(Configuration configuration) {
|
||||
return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null);
|
||||
interceptor, startLogErrorsCnt, connectionConf.getRpcTimeout(), metrics);
|
||||
}
|
||||
|
||||
public static RpcRetryingCallerFactory instantiate(Configuration configuration,
|
||||
ServerStatisticTracker stats) {
|
||||
return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, stats);
|
||||
MetricsConnection metrics) {
|
||||
return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null,
|
||||
metrics);
|
||||
}
|
||||
|
||||
public static RpcRetryingCallerFactory instantiate(Configuration configuration,
|
||||
RetryingCallerInterceptor interceptor, ServerStatisticTracker stats) {
|
||||
ServerStatisticTracker stats, MetricsConnection metrics) {
|
||||
return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, stats,
|
||||
metrics);
|
||||
}
|
||||
|
||||
public static RpcRetryingCallerFactory instantiate(Configuration configuration,
|
||||
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 });
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -644,7 +644,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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue