[ 
https://issues.apache.org/jira/browse/FLINK-10415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16630474#comment-16630474
 ] 

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_r220935608
 
 

 ##########
 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) {
 
 Review comment:
   We retry the send operation if we see an `IOException`. If any other 
exception occurs (e.g. `RuntimeException`), this might indicate that there is 
another problem where we don't want to retry. Therefore, I think it is ok to 
only catch the `IOException`. The request will also be failed if we see a 
different exception.

----------------------------------------------------------------
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:
us...@infra.apache.org


> 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)

Reply via email to