Skip to content

Commit

Permalink
HBASE-27273 Should stop autoRead and skip all the bytes when rpc requ…
Browse files Browse the repository at this point in the history
…est too big (#4679)

Signed-off-by: Xiaolin Ha <[email protected]>
(cherry picked from commit 486d19e)
  • Loading branch information
Apache9 committed Aug 6, 2022
1 parent 1183f06 commit 25cf86e
Showing 1 changed file with 12 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {

private final int maxFrameLength;
private boolean requestTooBig;
private boolean requestTooBigSent;
private String requestTooBigMessage;

public NettyRpcFrameDecoder(int maxFrameLength) {
Expand All @@ -58,8 +59,12 @@ void setConnection(NettyServerRpcConnection connection) {

@Override
protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
if (requestTooBigSent) {
in.skipBytes(in.readableBytes());
return;
}
if (requestTooBig) {
handleTooBigRequest(in);
handleTooBigRequest(ctx, in);
return;
}

Expand All @@ -83,7 +88,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) t
NettyRpcServer.LOG.warn(requestTooBigMessage);

if (connection.connectionHeaderRead) {
handleTooBigRequest(in);
handleTooBigRequest(ctx, in);
return;
}
ctx.channel().close();
Expand All @@ -101,7 +106,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) t
out.add(in.readRetainedSlice(frameLengthInt));
}

private void handleTooBigRequest(ByteBuf in) throws IOException {
private void handleTooBigRequest(ChannelHandlerContext ctx, ByteBuf in) throws IOException {
in.skipBytes(FRAME_LENGTH_FIELD_LENGTH);
in.markReaderIndex();
int preIndex = in.readerIndex();
Expand Down Expand Up @@ -146,6 +151,10 @@ private void handleTooBigRequest(ByteBuf in) throws IOException {
// instead of calling reqTooBig.sendResponseIfReady()
reqTooBig.param = null;
connection.channel.writeAndFlush(reqTooBig).addListener(ChannelFutureListener.CLOSE);
in.skipBytes(in.readableBytes());
requestTooBigSent = true;
// disable auto read as we do not care newer data from this channel any more
ctx.channel().config().setAutoRead(false);
}

private RPCProtos.RequestHeader getHeader(ByteBuf in, int headerSize) throws IOException {
Expand Down

0 comments on commit 25cf86e

Please sign in to comment.