HBASE-26170 handleTooBigRequest in NettyRpcServer didn't skip enough bytes (#3564)

Signed-off-by: stack <stack@apache.com>
This commit is contained in:
Xiaolin Ha 2021-08-05 23:05:26 +08:00 committed by GitHub
parent d4aed4d59e
commit c9383f2c89
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 16 additions and 13 deletions

View File

@ -87,7 +87,6 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {
NettyRpcServer.LOG.warn(requestTooBigMessage);
if (connection.connectionHeaderRead) {
in.skipBytes(FRAME_LENGTH_FIELD_LENGTH);
handleTooBigRequest(in);
return;
}
@ -107,6 +106,7 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {
}
private void handleTooBigRequest(ByteBuf in) throws IOException {
in.skipBytes(FRAME_LENGTH_FIELD_LENGTH);
in.markReaderIndex();
int preIndex = in.readerIndex();
int headerSize = readRawVarint32(in);
@ -118,6 +118,7 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {
}
if (in.readableBytes() < headerSize) {
NettyRpcServer.LOG.debug("headerSize is larger than readableBytes");
in.resetReaderIndex();
return;
}

View File

@ -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;
@ -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();
}
@ -64,17 +65,18 @@ public class TestRequestTooBigException {
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();