Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4569#discussion_r134471354
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientEndpoint.java
 ---
    @@ -155,38 +129,66 @@ public void shutdown() {
                        .set(HttpHeaders.Names.HOST, configuredTargetAddress + 
":" + configuredTargetPort)
                        .set(HttpHeaders.Names.CONNECTION, 
HttpHeaders.Values.CLOSE);
     
    -           synchronized (this) {
    -                   // This ensures strict sequential processing of 
requests.
    -                   // If we send new requests immediately we can no longer 
make assumptions about the order in which responses
    -                   // arrive, due to which the handler cannot know which 
future he should complete (not to mention what response
    -                   // type to read).
    -                   CompletableFuture<P> nextFuture = lastFuture
    -                           .handleAsync((f, e) -> 
submitRequest(httpRequest, messageHeaders), directExecutor)
    -                           .thenCompose((future) -> future);
    -
    -                   lastFuture = nextFuture;
    -                   return nextFuture;
    -           }
    +           return submitRequest(httpRequest, messageHeaders);
        }
     
        private <M extends MessageHeaders<R, P, U>, U extends ParameterMapper, 
R extends RequestBody, P extends ResponseBody> CompletableFuture<P> 
submitRequest(FullHttpRequest httpRequest, M messageHeaders) {
    -           CompletableFuture<P> responseFuture = 
handler.expectResponse(messageHeaders.getResponseClass());
    -
    -           try {
    -                   // write request
    -                   Channel channel = 
bootstrap.connect(configuredTargetAddress, 
configuredTargetPort).sync().channel();
    -                   channel.writeAndFlush(httpRequest);
    -                   channel.closeFuture();
    -           } catch (InterruptedException e) {
    -                   return FutureUtils.completedExceptionally(e);
    +           synchronized (lock) {
    +                   CompletableFuture<P> responseFuture = 
ClientHandler.addHandlerForResponse(bootstrap, sslEngine, 
messageHeaders.getResponseClass());
    +
    +                   try {
    +                           // write request
    +                           Channel channel = 
bootstrap.connect(configuredTargetAddress, 
configuredTargetPort).sync().channel();
    +                           channel.writeAndFlush(httpRequest);
    +                           channel.closeFuture();
    +                   } catch (InterruptedException e) {
    +                           return FutureUtils.completedExceptionally(e);
    +                   }
    +                   return responseFuture;
                }
    -           return responseFuture;
        }
     
    -   @ChannelHandler.Sharable
    -   private static class ClientHandler extends 
SimpleChannelInboundHandler<Object> {
    +   private static class RestChannelInitializer extends 
ChannelInitializer<SocketChannel> {
     
    -           private volatile ExpectedResponse<? extends ResponseBody> 
expectedResponse = null;
    +           private final SSLEngine sslEngine;
    +           private final ClientHandler handler;
    +
    +           public RestChannelInitializer(SSLEngine sslEngine, 
ClientHandler handler) {
    +                   this.sslEngine = sslEngine;
    +                   this.handler = handler;
    +           }
    +
    +           @Override
    +           protected void initChannel(SocketChannel ch) throws Exception {
    +                   // SSL should be the first handler in the pipeline
    +                   if (sslEngine != null) {
    +                           ch.pipeline().addLast("ssl", new 
SslHandler(sslEngine));
    +                   }
    +
    +                   ch.pipeline()
    +                           .addLast(new HttpClientCodec())
    +                           .addLast(new HttpObjectAggregator(1024 * 1024))
    +                           .addLast(handler)
    +                           .addLast(new PipelineErrorHandler(LOG));
    +           }
    +   }
    +
    +   private static class ClientHandler<P extends ResponseBody> extends 
SimpleChannelInboundHandler<Object> {
    +
    +           private final ExpectedResponse<P> expectedResponse;
    +
    +           private ClientHandler(ExpectedResponse<P> expectedResponse) {
    +                   this.expectedResponse = expectedResponse;
    +           }
    +
    +           static <P extends ResponseBody> CompletableFuture<P> 
addHandlerForResponse(Bootstrap bootStrap, SSLEngine sslEngine, Class<P> 
expectedResponse) {
    +                   CompletableFuture<P> responseFuture = new 
CompletableFuture<>();
    +
    +                   ClientHandler handler = new ClientHandler<>(new 
ExpectedResponse<>(expectedResponse, responseFuture));
    +                   bootStrap.handler(new RestChannelInitializer(sslEngine, 
handler));
    +
    +                   return responseFuture;
    +           }
     
                @Override
                protected void channelRead0(ChannelHandlerContext ctx, Object 
msg) throws Exception {
    --- End diff --
    
    Where do we actually close the connection?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to