HBASE-28741 Rpc ConnectionRegistry APIs should have timeout#8100
HBASE-28741 Rpc ConnectionRegistry APIs should have timeout#8100ndimiduk wants to merge 1 commit intoapache:masterfrom
Conversation
There was a problem hiding this comment.
Finally!
Looks good overall, just one thought - can we use a new config instead of using hbase.rpc.timeout? Because this is more of metadata level API than normal rpc API, we can consider a bit lower timeout as default value e.g. DEFAULT_HBASE_RPC_TIMEOUT / 6 or DEFAULT_HBASE_RPC_TIMEOUT / 3
| rpcClient = c; | ||
| addr2Stub = m; | ||
| addr2StubFuture.complete(m); | ||
| try { |
There was a problem hiding this comment.
Why we need the try catch here? FutureUtils.addListener does not throw any exception out.
| RpcChannel channel = rpcClient.createRpcChannel(server, user, 0); | ||
| // The preamble exchange still requires TCP connect and potentially TLS negotiation, so we use | ||
| // the configured rpc timeout to bound the connection attempt. | ||
| RpcChannel channel = rpcClient.createRpcChannel(server, user, rpcTimeoutMs); |
There was a problem hiding this comment.
We only send a preamble header here, I wonder whether the rpc timeout actually work. And I can make sure that we do not have TLS negotiation when sending preamble header.
| * promptly rather than returning a zombie future. | ||
| */ | ||
| @Test | ||
| public void testSubsequentCallsAfterFailure() throws Exception { |
There was a problem hiding this comment.
Does this test fail before the patch? We will create a ClusterIdFetcher every time when fetching in RpcConnectionRegistry, so I do not get the point why we could return a zombie future...
| public FailingRpcClientImpl(Configuration configuration, String clusterId, | ||
| SocketAddress localAddress, MetricsConnection metrics, Map<String, byte[]> attributes) { | ||
| if (clusterId != null) { | ||
| throw new RuntimeException("Simulated RPC client creation failure"); |
There was a problem hiding this comment.
So the actually problem is here, we may throw exception when creating RpcClient?
Looking at the code, neither BlockingRpcClient nor NettyRpcClient will throw exception when constructing.
@SuppressWarnings("FutureReturnValueIgnored")
public static <T> void addListener(CompletableFuture<T> future,
BiConsumer<? super T, ? super Throwable> action) {
future.whenComplete((resp, error) -> {
try {
// See this post on stack overflow(shorten since the url is too long),
// https://s.apache.org/completionexception
// For a chain of CompletableFuture, only the first child CompletableFuture can get the
// original exception, others will get a CompletionException, which wraps the original
// exception. So here we unwrap it before passing it to the callback action.
action.accept(resp, unwrapCompletionException(error));
} catch (Throwable t) {
LOG.error("Unexpected error caught when processing CompletableFuture", t);
}
});
}
We will log something out when the callback throws exception, have you seen something like this?
No description provided.