[ https://issues.apache.org/jira/browse/FLINK-10415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16628507#comment-16628507 ]
ASF GitHub Bot commented on FLINK-10415: ---------------------------------------- zentol 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_r220502083 ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java ########## @@ -89,7 +91,78 @@ public void testInvalidVersionRejection() throws Exception { } catch (IllegalArgumentException e) { // expected } + } + /** + * Tests that we fail the operation if the remote connection closes. + */ + @Test + public void testConnectionClosedHandling() throws Exception { + final ServerSocket serverSocket = new ServerSocket(0); + final String targetAddress = "localhost"; + final int targetPort = serverSocket.getLocalPort(); + + try (final RestClient restClient = new RestClient(RestClientConfiguration.fromConfiguration(new Configuration()), TestingUtils.defaultExecutor())) { + final CompletableFuture<EmptyResponseBody> responseFuture = restClient.sendRequest( + targetAddress, + targetPort, + new TestMessageHeaders(), + EmptyMessageParameters.getInstance(), + EmptyRequestBody.getInstance(), + Collections.emptyList()); + + // establish connection + final Socket connectionSocket = serverSocket.accept(); + + // close connection + connectionSocket.close(); + + try { + responseFuture.get(); + } catch (ExecutionException ee) { + assertThat(ExceptionUtils.findThrowable(ee, ConnectionClosedException.class).isPresent(), is(true)); + } + } finally { + serverSocket.close(); + } + } + + /** + * Tests that we fail the operation if the client closes. + */ + @Test + public void testRestClientClosedHandling() throws Exception { + final ServerSocket serverSocket = new ServerSocket(0); + final String targetAddress = "localhost"; + final int targetPort = serverSocket.getLocalPort(); + Socket connectionSocket = null; + + try (final RestClient restClient = new RestClient(RestClientConfiguration.fromConfiguration(new Configuration()), TestingUtils.defaultExecutor())) { + final CompletableFuture<EmptyResponseBody> responseFuture = restClient.sendRequest( Review comment: I'm wondering...consider the `submitRequest` method from the `RestClient`: ``` private <P extends ResponseBody> CompletableFuture<P> submitRequest(String targetAddress, int targetPort, Request httpRequest, JavaType responseType) { 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()); } }); return channelFuture .thenComposeAsync( channel -> { ClientHandler handler = channel.pipeline().get(ClientHandler.class); CompletableFuture<JsonResponse> future = handler.getJsonFuture(); try { httpRequest.writeTo(channel); } catch (IOException e) { return FutureUtils.completedExceptionally(new FlinkException("Could not write request.", e)); } return future; }, executor) .thenComposeAsync( (JsonResponse rawResponse) -> parseResponse(rawResponse, responseType), executor); ``` What happens if the client is shutdown, before the `thenComposeAsync` portions have been run? Then the channelFuture is completed by the listener; and then subsequent stages aren't executed? ---------------------------------------------------------------- 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)