You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2023/09/20 13:56:47 UTC
[hbase] branch branch-2 updated: HBASE-28101 Should check the return value of protobuf Message.mergeDelimitedFrom (#5413)
This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2 by this push:
new ddb297bca66 HBASE-28101 Should check the return value of protobuf Message.mergeDelimitedFrom (#5413)
ddb297bca66 is described below
commit ddb297bca660852272a2e843c66ffa0453fba48d
Author: Duo Zhang <zh...@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 <hu...@apache.org>
(cherry picked from commit 93d90bf64d6dfbdfbdd712f2748857ae282a3014)
---
.../java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java | 8 +++++++-
1 file changed, 7 insertions(+), 1 deletion(-)
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 700093a3027..34869314bab 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
@@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.ipc;
import io.opentelemetry.context.Scope;
+import java.io.EOFException;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
@@ -168,7 +169,12 @@ class NettyRpcDuplexHandler extends ChannelDuplexHandler {
Message value;
if (call.responseDefaultType != null) {
Message.Builder builder = call.responseDefaultType.newBuilderForType();
- builder.mergeDelimitedFrom(in);
+ 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;