Skip to content

Commit

Permalink
HBASE-28101 Should check the return value of protobuf Message.mergeDe…
Browse files Browse the repository at this point in the history
…limitedFrom (apache#5413)

Signed-off-by: GeorryHuang <[email protected]>
(cherry picked from commit 93d90bf)
  • Loading branch information
Apache9 committed Sep 20, 2023
1 parent 6cac6c2 commit d74583c
Showing 1 changed file with 7 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -168,7 +169,12 @@ private void readResponse(ChannelHandlerContext ctx, ByteBuf buf) throws IOExcep
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;
Expand Down

0 comments on commit d74583c

Please sign in to comment.