[
https://issues.apache.org/jira/browse/FLINK-10415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16630478#comment-16630478
]
ASF GitHub Bot commented on FLINK-10415:
----------------------------------------
tillrohrmann commented on a change in pull request #6763: [FLINK-10415] Fail
response future if connection closes in RestClient
URL: https://github.com/apache/flink/pull/6763#discussion_r220936630
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java
##########
@@ -339,12 +338,26 @@ private static Request createRequest(String
targetAddress, String targetUrl, Htt
.thenComposeAsync(
channel -> {
ClientHandler handler =
channel.pipeline().get(ClientHandler.class);
- CompletableFuture<JsonResponse> future
= handler.getJsonFuture();
+
+ CompletableFuture<JsonResponse> future;
+ boolean success = false;
+
try {
- httpRequest.writeTo(channel);
+ if (handler == null) {
+ throw new
IOException("Netty pipeline was not properly initialized.");
+ } else {
+
httpRequest.writeTo(channel);
+ future =
handler.getJsonFuture();
+ success = true;
+ }
} catch (IOException e) {
- return
FutureUtils.completedExceptionally(new FlinkException("Could not write
request.", e));
+ future =
FutureUtils.completedExceptionally(new ConnectionException("Could not write
request.", e));
+ } finally {
+ if (!success) {
+ channel.close();
Review comment:
We keep the channel open if we managed to send the request to the remote
endpoint, because we expect an answer. In all other cases (`!success`) we
should close it, because we cannot expect something to return.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> RestClient does not react to lost connection
> --------------------------------------------
>
> Key: FLINK-10415
> URL: https://issues.apache.org/jira/browse/FLINK-10415
> Project: Flink
> Issue Type: Bug
> Components: REST
> Affects Versions: 1.6.1, 1.7.0, 1.5.4
> Reporter: Till Rohrmann
> Assignee: Till Rohrmann
> Priority: Blocker
> Labels: pull-request-available
> Fix For: 1.7.0, 1.6.2, 1.5.5
>
>
> While working on FLINK-10403, I noticed that Flink's {{RestClient}} does not
> seem to react to a lost connections in time. When sending a request to the
> current leader it happened that the leader was killed just after establishing
> the connection. Then the {{RestClient}} did not fail the connection and was
> stuck in writing a request or retrieving a response from the lost leader. I'm
> wondering whether we should introduce a {{ReadTimeoutHandler}} and
> {{WriteTimeoutHandler}} to handle these problems.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)