This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new 2ea2f80  HBASE-26170 handleTooBigRequest in NettyRpcServer didn't skip 
enough bytes (#3564)
2ea2f80 is described below

commit 2ea2f800b05c15b478d5e077148e34e4145cff09
Author: Xiaolin Ha <haxiao...@apache.org>
AuthorDate: Thu Aug 5 23:05:26 2021 +0800

    HBASE-26170 handleTooBigRequest in NettyRpcServer didn't skip enough bytes 
(#3564)
    
    Signed-off-by: stack <st...@apache.com>
---
 .../hadoop/hbase/ipc/NettyRpcFrameDecoder.java     |  3 ++-
 .../hbase/client/TestRequestTooBigException.java   | 26 ++++++++++++----------
 2 files changed, 16 insertions(+), 13 deletions(-)

diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcFrameDecoder.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcFrameDecoder.java
index c46fe78..6891243 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcFrameDecoder.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcFrameDecoder.java
@@ -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;
     }
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestTooBigException.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestTooBigException.java
index 5f7f154..f81b3d4 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestTooBigException.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestTooBigException.java
@@ -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.HBaseTestingUtility;
 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();

Reply via email to