This is an automated email from the ASF dual-hosted git repository. aleksey pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra.git
commit e572c8fca0c5cd68229b8db8d4915817d5d49daf Author: Aleksey Yeshchenko <alek...@apache.org> AuthorDate: Mon Nov 16 17:36:24 2020 +0000 'SSLEngine closed already' exception on failed outbound connection patch by Aleksey Yeschenko; reviewed by Alex Petrov and Norman Maurer for (CASSANDRA-16277) --- CHANGES.txt | 1 + .../cassandra/net/OutboundConnectionInitiator.java | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 176bc04..7d904a9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0-beta4 + * 'SSLEngine closed already' exception on failed outbound connection (CASSANDRA-16277) * Drain and/or shutdown might throw because of slow messaging service shutdown (CASSANDRA-16276) * Upgrade JNA to 5.6.0, dropping support for <=glibc-2.6 systems (CASSANDRA-16212) * Add saved Host IDs to TokenMetadata at startup (CASSANDRA-16246) diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java index 4a5585a..2c26005 100644 --- a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java +++ b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java @@ -42,6 +42,7 @@ import io.netty.handler.codec.ByteToMessageDecoder; import io.netty.handler.logging.LogLevel; import io.netty.handler.logging.LoggingHandler; +import io.netty.handler.ssl.SslClosedEngineException; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; import io.netty.util.concurrent.FailedFuture; @@ -86,6 +87,7 @@ public class OutboundConnectionInitiator<SuccessType extends OutboundConnectionI private final OutboundConnectionSettings settings; private final int requestMessagingVersion; // for pre40 nodes private final Promise<Result<SuccessType>> resultPromise; + private boolean isClosed; private OutboundConnectionInitiator(ConnectionType type, OutboundConnectionSettings settings, int requestMessagingVersion, Promise<Result<SuccessType>> resultPromise) @@ -363,6 +365,21 @@ public class OutboundConnectionInitiator<SuccessType extends OutboundConnectionI @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + if (isClosed && cause instanceof SslClosedEngineException) + { + /* + * Occasionally Netty will invoke this handler to process an exception of the following kind: + * io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer + * + * When we invoke ctx.close() later in this method, the listener, set up in channelActive(), might be + * failed with an SslClosedEngineException("SSLEngine closed already") by Netty, and exceptionCaught() will be invoked + * once again, this time to handle the SSLException triggered by ctx.close(). + * + * The exception at this stage is benign, and we shouldn't be double-logging the failure to connect. + */ + return; + } + try { JVMStabilityInspector.inspectThrowable(cause, false); @@ -371,6 +388,7 @@ public class OutboundConnectionInitiator<SuccessType extends OutboundConnectionI logger.info("Failed to connect to peer {}", settings.connectToId(), cause); else logger.error("Failed to handshake with peer {}", settings.connectToId(), cause); + isClosed = true; ctx.close(); } catch (Throwable t) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org