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

ASF GitHub Bot commented on FLINK-10309:
----------------------------------------

zentol commented on a change in pull request #6785: [FLINK-10309][rest] Before 
shutting down cluster, wait for asynchronous operations
URL: https://github.com/apache/flink/pull/6785#discussion_r223364545
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
 ##########
 @@ -75,19 +82,19 @@ protected AbstractRestHandler(
 
        @Override
        protected CompletableFuture<Void> 
respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, 
HandlerRequest<R, M> handlerRequest, T gateway) {
-               CompletableFuture<P> response;
+               final CompletableFuture<P> response = 
handleRequestInternal(handlerRequest, gateway);
 
-               try {
-                       response = handleRequest(handlerRequest, gateway);
-               } catch (RestHandlerException e) {
-                       response = FutureUtils.completedExceptionally(e);
-               }
+               return response.whenCompleteAsync((P resp, Throwable throwable) 
-> {
+                               inFlightRequestTracker.deregisterRequest();
 
-               return response.whenComplete((P resp, Throwable throwable) -> {
-                       Tuple2<ResponseBody, HttpResponseStatus> r = throwable 
!= null ?
-                               errorResponse(throwable) : Tuple2.of(resp, 
messageHeaders.getResponseStatusCode());
-                       HandlerUtils.sendResponse(ctx, httpRequest, r.f0, r.f1, 
responseHeaders);
-               }).thenApply(ignored -> null);
+                               Tuple2<ResponseBody, HttpResponseStatus> r = 
throwable != null ?
+                                       errorResponse(throwable) : 
Tuple2.of(resp, messageHeaders.getResponseStatusCode());
+                               HandlerUtils.sendResponse(ctx, httpRequest, 
r.f0, r.f1, responseHeaders);
+                       },
+                       // send response from Netty's EventExecutor so that the 
server channel
+                       // does not close while we are responding to in-flight 
requests
+                       ctx.executor())
 
 Review comment:
   under what circumstances would the server channel close? Does it simply 
close after a set or timeout or ...?

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


> Cancel flink job occurs java.net.ConnectException
> -------------------------------------------------
>
>                 Key: FLINK-10309
>                 URL: https://issues.apache.org/jira/browse/FLINK-10309
>             Project: Flink
>          Issue Type: Bug
>          Components: Distributed Coordination, REST
>    Affects Versions: 1.5.3, 1.6.0, 1.7.0
>            Reporter: vinoyang
>            Assignee: Gary Yao
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 1.7.0, 1.6.2, 1.5.5
>
>
> The problem occurs when using the Yarn per-job detached mode. Trying to 
> cancel with savepoint fails with the following exception before being able to 
> retrieve the savepoint path:
> exception stack trace : 
> {code:java}
> org.apache.flink.util.FlinkException: Could not cancel job xxxx.
>         at 
> org.apache.flink.client.cli.CliFrontend.lambda$cancel$4(CliFrontend.java:585)
>         at 
> org.apache.flink.client.cli.CliFrontend.runClusterAction(CliFrontend.java:960)
>         at 
> org.apache.flink.client.cli.CliFrontend.cancel(CliFrontend.java:577)
>         at 
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:1034)
>         at java.lang.Thread.run(Thread.java:748)
> Caused by: java.util.concurrent.ExecutionException: 
> org.apache.flink.runtime.concurrent.FutureUtils$RetryException: Could not 
> complete the operation. Number of retries has been exhausted.
>         at 
> java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
>         at 
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
>         at 
> org.apache.flink.client.program.rest.RestClusterClient.cancelWithSavepoint(RestClusterClient.java:398)
>         at 
> org.apache.flink.client.cli.CliFrontend.lambda$cancel$4(CliFrontend.java:583)
>         ... 6 more
> Caused by: org.apache.flink.runtime.concurrent.FutureUtils$RetryException: 
> Could not complete the operation. Number of retries has been exhausted.
>         at 
> org.apache.flink.runtime.concurrent.FutureUtils.lambda$retryOperationWithDelay$5(FutureUtils.java:213)
>         at 
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
>         at 
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
>         at 
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
>         at 
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
>         at 
> org.apache.flink.runtime.rest.RestClient.lambda$submitRequest$1(RestClient.java:274)
>         at 
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:680)
>         at 
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:603)
>         at 
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:563)
>         at 
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:424)
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.fulfillConnectPromise(AbstractNioChannel.java:268)
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:284)
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:528)
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>         at 
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>         at 
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
>         ... 1 more
> Caused by: java.util.concurrent.CompletionException: 
> java.net.ConnectException: Connect refuse: xxx/xxx.xxx.xxx.xxx:xxx
>         at 
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>         at 
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>         at 
> java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:943)
>         at 
> java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:926)
>         ... 16 more
> Caused by: java.net.ConnectException: Connect refuse: xxx/xxx.xxx.xxx.xxx:xxx
>         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>         at 
> sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:224)
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:281)
>         ... 7 more
> {code}
> some discussion in mailing list : 
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/Cancel-flink-job-occur-exception-td24056.html



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to