Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-25933

Log trace raw exception, instead of cause message in NettyRpcServerRequestDecoder

VotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • 3.0.0-alpha-1, 2.2.7, 2.5.0, 2.3.5, 2.4.3
    • 3.0.0-alpha-1, 2.5.0, 2.3.6, 2.4.4
    • None
    • None

    Description

      In NettyRpcServerRequestDecoder, override of exceptionCaught method tries to log the exception cause message, however this not always will have a content, causing debugging of connection failure issues harder to troubleshoot. We should simply log trace the observed Throwable reference itself:

       

      diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
      index 1e844bb02cb..40f59ad1259 100644
      --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
      +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
      @@ -74,7 +74,7 @@ class NettyRpcServerRequestDecoder extends ChannelInboundHandlerAdapter {
         public void exceptionCaught(ChannelHandlerContext ctx, Throwable e) {
           allChannels.remove(ctx.channel());
           NettyRpcServer.LOG.trace("Connection {}; caught unexpected downstream exception.",
      -        ctx.channel().remoteAddress(), e.getCause());
      +        ctx.channel().remoteAddress(), e);
           ctx.channel().close();
         }
       }
       

       

      Attachments

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            wchevreuil Wellington Chevreuil
            wchevreuil Wellington Chevreuil
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment