Patrick Lucas created FLINK-32583:
-------------------------------------
Summary: RestClient can deadlock if request made after Netty event
executor terminated
Key: FLINK-32583
URL: https://issues.apache.org/jira/browse/FLINK-32583
Project: Flink
Issue Type: Bug
Components: Runtime / REST
Affects Versions: 1.18.0, 1.16.3, 1.17.2
Reporter: Patrick Lucas
The RestClient can deadlock if a request is made after the Netty event executor
has terminated.
This is due to the listener that would resolve the CompletableFuture that is
attached to the ChannelFuture returned by the call to Netty to connect not
being able to run because the executor to run it rejects the execution.
[RestClient.java|https://github.com/apache/flink/blob/release-1.17.1/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java#L471-L482]:
{code:java}
final ChannelFuture connectFuture = bootstrap.connect(targetAddress,
targetPort);
final CompletableFuture<Channel> channelFuture = new CompletableFuture<>();
connectFuture.addListener(
(ChannelFuture future) -> {
if (future.isSuccess()) {
channelFuture.complete(future.channel());
} else {
channelFuture.completeExceptionally(future.cause());
}
});
{code}
In this code, the call to {{addListener()}} can fail silently (only logging to
the console), meaning any code waiting on the CompletableFuture returned by
this method will deadlock.
There was some work in Netty around this back in 2015, but it's unclear to me
how this situation is expected to be handled given the discussion and changes
from these issues:
* [https://github.com/netty/netty/issues/3449] (open)
* [https://github.com/netty/netty/pull/3483] (closed)
* [https://github.com/netty/netty/pull/3566] (closed)
* [https://github.com/netty/netty/pull/5087] (merged)
I think a reasonable fix for Flink would be to check the state of
{{connectFuture}} and {{channelFuture}} immediately after the call to
{{addListener()}}, resolving {{channelFuture}} with {{completeExceptionally()}}
if {{connectFuture}} is done and failed and {{channelFuture}} has not been
completed. In the possible race condition where the listener was attached
successfully and the connection fails instantly, the result is the same, as
calls to {{CompletableFuture#completeExceptionally()}} are idempotent.
A workaround for users of RestClient is to call {{CompletableFuture#get(long
timeout, TimeUnit unit)}} rather than {{#get()}} or {{#join()}} on the
CompletableFutures it returns. However, if the call throws TimeoutException,
the cause of the failure cannot easily be determined.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)