HBASE-26170 handleTooBigRequest in NettyRpcServer didn't skip enough bytes (#3564)
Signed-off-by: stack <stack@apache.com>
This commit is contained in:
parent
d4aed4d59e
commit
c9383f2c89
|
@ -87,7 +87,6 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {
|
||||||
NettyRpcServer.LOG.warn(requestTooBigMessage);
|
NettyRpcServer.LOG.warn(requestTooBigMessage);
|
||||||
|
|
||||||
if (connection.connectionHeaderRead) {
|
if (connection.connectionHeaderRead) {
|
||||||
in.skipBytes(FRAME_LENGTH_FIELD_LENGTH);
|
|
||||||
handleTooBigRequest(in);
|
handleTooBigRequest(in);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -107,6 +106,7 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void handleTooBigRequest(ByteBuf in) throws IOException {
|
private void handleTooBigRequest(ByteBuf in) throws IOException {
|
||||||
|
in.skipBytes(FRAME_LENGTH_FIELD_LENGTH);
|
||||||
in.markReaderIndex();
|
in.markReaderIndex();
|
||||||
int preIndex = in.readerIndex();
|
int preIndex = in.readerIndex();
|
||||||
int headerSize = readRawVarint32(in);
|
int headerSize = readRawVarint32(in);
|
||||||
|
@ -118,6 +118,7 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (in.readableBytes() < headerSize) {
|
if (in.readableBytes() < headerSize) {
|
||||||
|
NettyRpcServer.LOG.debug("headerSize is larger than readableBytes");
|
||||||
in.resetReaderIndex();
|
in.resetReaderIndex();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,8 +17,8 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hbase.ipc.RpcServer.MAX_REQUEST_SIZE;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtil;
|
import org.apache.hadoop.hbase.HBaseTestingUtil;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
@ -48,6 +48,7 @@ public class TestRequestTooBigException {
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUpBeforeClass() throws Exception {
|
public static void setUpBeforeClass() throws Exception {
|
||||||
|
TEST_UTIL.getConfiguration().setInt(MAX_REQUEST_SIZE, 10000);
|
||||||
TEST_UTIL.startMiniCluster();
|
TEST_UTIL.startMiniCluster();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -64,17 +65,18 @@ public class TestRequestTooBigException {
|
||||||
TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
|
TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
|
||||||
try {
|
try {
|
||||||
byte[] value = new byte[2 * 2014 * 1024];
|
byte[] value = new byte[2 * 2014 * 1024];
|
||||||
|
for (int m = 0; m < 10000; m++) {
|
||||||
Put p = new Put(Bytes.toBytes("bigrow"));
|
Put p = new Put(Bytes.toBytes("bigrow"));
|
||||||
// big request = 400*2 M
|
// big request = 400*2 M
|
||||||
for (int i = 0; i < 400; i++) {
|
for (int i = 0; i < 400; i++) {
|
||||||
p.addColumn(family, Bytes.toBytes("someQualifier" + i), value);
|
p.addColumn(family, Bytes.toBytes("someQualifier" + i), value);
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
table.put(p);
|
table.put(p);
|
||||||
assertTrue("expected RequestTooBigException", false);
|
assertTrue("expected RequestTooBigException", false);
|
||||||
} catch (RequestTooBigException e) {
|
} catch (RequestTooBigException e) {
|
||||||
assertTrue("expected RequestTooBigException", true);
|
assertTrue("expected RequestTooBigException", true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
table.close();
|
table.close();
|
||||||
|
|
Loading…
Reference in New Issue