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

Duo Zhang commented on HBASE-27782:
-----------------------------------

OK, checkd the code in netty.

{code}
    private void handleUnwrapThrowable(ChannelHandlerContext ctx, Throwable 
cause) {
        try {
            // We should attempt to notify the handshake failure before writing 
any pending data. If we are in unwrap
            // and failed during the handshake process, and we attempt to wrap, 
then promises will fail, and if
            // listeners immediately close the Channel then we may end up 
firing the handshake event after the Channel
            // has been closed.
            if (handshakePromise.tryFailure(cause)) {
                ctx.fireUserEventTriggered(new 
SslHandshakeCompletionEvent(cause));
            }

            // Let's check if the handler was removed in the meantime and so 
pendingUnencryptedWrites is null.
            if (pendingUnencryptedWrites != null) {
                // We need to flush one time as there may be an alert that we 
should send to the remote peer because
                // of the SSLException reported here.
                wrapAndFlush(ctx);
            }
        } catch (SSLException ex) {
            logger.debug("SSLException during trying to call 
SSLEngine.wrap(...)" +
                    " because of an previous SSLException, ignoring...", ex);
        } finally {
            // ensure we always flush and close the channel.
            setHandshakeFailure(ctx, cause, true, false, true);
        }
        PlatformDependent.throwException(cause); <==== should be this line
    }
{code}

We will enter the handleUnwrapThrowable but at the last of this method, we will 
throw the exception out...

It should not have any real problems as in the beginning of the method, they 
marked the handshakePromise to fail so in our NettyRpcConnection we can receive 
the exception and close the connection.

{code}
        @Override
        public void operationComplete(ChannelFuture future) throws Exception {
          Channel ch = future.channel();
          if (!future.isSuccess()) {
            fail(ch, future.cause());
            return;
          }
          SslHandler sslHandler = ch.pipeline().get(SslHandler.class);
          if (sslHandler != null) {
            NettyFutureUtils.addListener(sslHandler.handshakeFuture(), f -> {
              if (f.isSuccess()) {
                succeed(ch);
              } else {
                fail(ch, f.cause());
              }
            });
          } else {
            succeed(ch);
          }
        }
{code}

Let me see if I can reproduce the problem with a UT. And provide a patch to 
avoid the confusing log output.

Thanks.

> During SSL handshake error, netty complains that exceptionCaught() was not 
> handled
> ----------------------------------------------------------------------------------
>
>                 Key: HBASE-27782
>                 URL: https://issues.apache.org/jira/browse/HBASE-27782
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Bryan Beaudreault
>            Priority: Major
>              Labels: TLS
>
> I was chaos testing the new native TLS, forcing a certificate to expire and 
> fail handshake. The handshake failure properly causes submitted requests to 
> fail, but I see the following "unhandled exception" like message:
> {code:java}
> WARN  o.a.h.t.i.n.c.DefaultChannelPipeline - An exceptionCaught() event was 
> fired, and it reached at the tail of the pipeline. It usually means the last 
> handler in the pipeline did not handle the exception.
> org.apache.hbase.thirdparty.io.netty.handler.codec.DecoderException: 
> javax.net.ssl.SSLHandshakeException: Received fatal alert: certificate_expired
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:499)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:290)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>         at java.base/java.lang.Thread.run(Thread.java:833)
> Caused by: javax.net.ssl.SSLHandshakeException: Received fatal alert: 
> certificate_expired
>         at java.base/sun.security.ssl.Alert.createSSLException(Alert.java:131)
>         at java.base/sun.security.ssl.Alert.createSSLException(Alert.java:117)
>         at 
> java.base/sun.security.ssl.TransportContext.fatal(TransportContext.java:358)
>         at 
> java.base/sun.security.ssl.Alert$AlertConsumer.consume(Alert.java:293)
>         at 
> java.base/sun.security.ssl.TransportContext.dispatch(TransportContext.java:204)
>         at 
> java.base/sun.security.ssl.SSLTransport.decode(SSLTransport.java:172)
>         at 
> java.base/sun.security.ssl.SSLEngineImpl.decode(SSLEngineImpl.java:736)
>         at 
> java.base/sun.security.ssl.SSLEngineImpl.readRecord(SSLEngineImpl.java:691)
>         at 
> java.base/sun.security.ssl.SSLEngineImpl.unwrap(SSLEngineImpl.java:506)
>         at 
> java.base/sun.security.ssl.SSLEngineImpl.unwrap(SSLEngineImpl.java:482)
>         at java.base/javax.net.ssl.SSLEngine.unwrap(SSLEngine.java:679)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.ssl.SslHandler$SslEngineType$3.unwrap(SslHandler.java:296)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.ssl.SslHandler.unwrap(SslHandler.java:1343)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.ssl.SslHandler.decodeJdkCompatible(SslHandler.java:1236)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.ssl.SslHandler.decode(SslHandler.java:1285)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:529)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:468)
>         ... 17 common frames omitted{code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to