Skip to content

Commit

Permalink
HBASE-26170 handleTooBigRequest in NettyRpcServer didn't skip enough …
Browse files Browse the repository at this point in the history
…bytes (#3564)

Signed-off-by: stack <stack@apache.com>
  • Loading branch information
sunhelly authored Aug 5, 2021
1 parent d4aed4d commit c9383f2
Show file tree
Hide file tree
Showing 2 changed files with 16 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,6 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out)
NettyRpcServer.LOG.warn(requestTooBigMessage);

if (connection.connectionHeaderRead) {
in.skipBytes(FRAME_LENGTH_FIELD_LENGTH);
handleTooBigRequest(in);
return;
}
Expand All @@ -107,6 +106,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out)
}

private void handleTooBigRequest(ByteBuf in) throws IOException {
in.skipBytes(FRAME_LENGTH_FIELD_LENGTH);
in.markReaderIndex();
int preIndex = in.readerIndex();
int headerSize = readRawVarint32(in);
Expand All @@ -118,6 +118,7 @@ private void handleTooBigRequest(ByteBuf in) throws IOException {
}

if (in.readableBytes() < headerSize) {
NettyRpcServer.LOG.debug("headerSize is larger than readableBytes");
in.resetReaderIndex();
return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hbase.client;

import static org.apache.hadoop.hbase.ipc.RpcServer.MAX_REQUEST_SIZE;
import static org.junit.Assert.assertTrue;

import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtil;
import org.apache.hadoop.hbase.TableName;
Expand Down Expand Up @@ -48,6 +48,7 @@ public class TestRequestTooBigException {

@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().setInt(MAX_REQUEST_SIZE, 10000);
TEST_UTIL.startMiniCluster();
}

Expand All @@ -64,17 +65,18 @@ public void testHbasePutDeleteCell() throws Exception {
TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
try {
byte[] value = new byte[2 * 2014 * 1024];

Put p = new Put(Bytes.toBytes("bigrow"));
// big request = 400*2 M
for (int i = 0; i < 400; i++) {
p.addColumn(family, Bytes.toBytes("someQualifier" + i), value);
}
try {
table.put(p);
assertTrue("expected RequestTooBigException", false);
} catch (RequestTooBigException e) {
assertTrue("expected RequestTooBigException", true);
for (int m = 0; m < 10000; m++) {
Put p = new Put(Bytes.toBytes("bigrow"));
// big request = 400*2 M
for (int i = 0; i < 400; i++) {
p.addColumn(family, Bytes.toBytes("someQualifier" + i), value);
}
try {
table.put(p);
assertTrue("expected RequestTooBigException", false);
} catch (RequestTooBigException e) {
assertTrue("expected RequestTooBigException", true);
}
}
} finally {
table.close();
Expand Down

0 comments on commit c9383f2

Please sign in to comment.