kezhuw commented on code in PR #1917: URL: https://github.com/apache/zookeeper/pull/1917#discussion_r1198737024
########## zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java: ########## @@ -196,13 +209,26 @@ public void operationComplete(ChannelFuture channelFuture) throws Exception { } } + boolean cancelled(ChannelFuture channelFuture) { + if (connectFuture == null) { + LOG.info("connect attempt cancelled"); + // If the connect attempt was cancelled but succeeded + // anyway, make sure to close the channel, otherwise + // we may leak a file descriptor. + channelFuture.channel().close(); + return true; + } + return false; + } + @Override void cleanup() { connectLock.lock(); try { if (connectFuture != null) { connectFuture.cancel(false); connectFuture = null; + afterConnectFutureCancel(); } if (channel != null) { channel.close().syncUninterruptibly(); Review Comment: I think it is a good to move this blocking `syncUninterruptibly` part out of lock scope. This could also solve the deadlock issue. ########## zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java: ########## @@ -196,13 +209,26 @@ public void operationComplete(ChannelFuture channelFuture) throws Exception { } } + boolean cancelled(ChannelFuture channelFuture) { Review Comment: I think we could concentrate more on identity of `channelFuture` and `connectFuture`. This will make it clear that `channelFuture` could come from a old connection. Better to document a bit. It is also possible that new connection is not completed yet. In this case, old connection should be simply filtered out, otherwise we could introduce inconsistence with outside world. ########## zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java: ########## @@ -196,13 +209,26 @@ public void operationComplete(ChannelFuture channelFuture) throws Exception { } } + boolean cancelled(ChannelFuture channelFuture) { + if (connectFuture == null) { + LOG.info("connect attempt cancelled"); + // If the connect attempt was cancelled but succeeded + // anyway, make sure to close the channel, otherwise + // we may leak a file descriptor. + channelFuture.channel().close(); + return true; + } + return false; + } + @Override void cleanup() { connectLock.lock(); try { if (connectFuture != null) { connectFuture.cancel(false); Review Comment: After taking a look at netty code [AbstractNioChannel::connect](https://github.com/netty/netty/blob/4.1/transport/src/main/java/io/netty/channel/nio/AbstractNioChannel.java#L235), [AbstractEpollChannel::connect](https://github.com/netty/netty/blob/4.1/transport-classes-epoll/src/main/java/io/netty/channel/epoll/AbstractEpollChannel.java#L588) and [AbstractKQueueChannel::connect](https://github.com/netty/netty/blob/4.1/transport-classes-kqueue/src/main/java/io/netty/channel/kqueue/AbstractKQueueChannel.java#L546), I think this statement `connectFuture.cancel(false)` cloud be the root cause. Basically, a connecting future in uncancellable which means `connectFuture.cancel(false)` could be a nop. So, the old `connectFuture` could still pending for completion. When we cleanup new `connectFuture` and old `connectFuture` completes, they are stuck due to leak of event threads to complete `channel.close().syncUninterruptibly()`(invoke close in event thread). I think we could also wait `connectFuture` to complete in `cleanup`, but I don't think it is a good. Connection timeout is a possible reason for us to fall here. It is nonsense to block next connection for the same reason. I have watched this issue for a while, and could not image a situation where there is single `connectFuture` and this stuck emerged. The `channel.close().syncUninterruptibly()` means its `connectFuture` already completed successfully. https://github.com/apache/zookeeper/blob/a64dbf5b06ca1a73dc2ad6c7d31e679e312082aa/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java#L86 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: notifications-unsubscr...@zookeeper.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org