This is an automated email from the ASF dual-hosted git repository. zhangduo pushed a commit to branch branch-2.4 in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2.4 by this push: new 672861c821f HBASE-28101 Should check the return value of protobuf Message.mergeDelimitedFrom (#5413) 672861c821f is described below commit 672861c821f766d50fc41c7114e03fb05fcd1580 Author: Duo Zhang <zhang...@apache.org> AuthorDate: Wed Sep 20 21:39:16 2023 +0800 HBASE-28101 Should check the return value of protobuf Message.mergeDelimitedFrom (#5413) Signed-off-by: GeorryHuang <huangzhuo...@apache.org> (cherry picked from commit 93d90bf64d6dfbdfbdd712f2748857ae282a3014) --- .../org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java index ef7aed4889e..bd2ca74bb9c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.ipc; +import java.io.EOFException; import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -30,7 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.protobuf.Message; -import org.apache.hbase.thirdparty.com.google.protobuf.Message.Builder; import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufInputStream; @@ -164,8 +164,13 @@ class NettyRpcDuplexHandler extends ChannelDuplexHandler { } Message value; if (call.responseDefaultType != null) { - Builder builder = call.responseDefaultType.newBuilderForType(); - builder.mergeDelimitedFrom(in); + Message.Builder builder = call.responseDefaultType.newBuilderForType(); + if (!builder.mergeDelimitedFrom(in)) { + // The javadoc of mergeDelimitedFrom says returning false means the stream reaches EOF + // before reading any bytes out, so here we need to manually throw the EOFException out + throw new EOFException( + "EOF while reading response with type: " + call.responseDefaultType.getClass().getName()); + } value = builder.build(); } else { value = null;