You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2021/08/07 15:23:05 UTC
[hbase] branch branch-2.3 updated: HBASE-26179
TestRequestTooBigException spends too much time to finish (#3571)
This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2.3 by this push:
new 044892f HBASE-26179 TestRequestTooBigException spends too much time to finish (#3571)
044892f is described below
commit 044892f23ea3e76a48252b939155d66176ba7c44
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Sat Aug 7 23:01:55 2021 +0800
HBASE-26179 TestRequestTooBigException spends too much time to finish (#3571)
Signed-off-by: Yulin Niu <ni...@apache.org>
---
.../hadoop/hbase/ipc/NettyRpcFrameDecoder.java | 9 ++--
.../org/apache/hadoop/hbase/ipc/RpcServer.java | 4 --
.../hbase/ipc/SimpleServerRpcConnection.java | 12 +++--
.../hbase/client/TestRequestTooBigException.java | 57 ++++++++++------------
4 files changed, 38 insertions(+), 44 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 6891243..01cf9b5 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
@@ -130,16 +130,17 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {
NettyServerCall reqTooBig = connection.createCall(header.getCallId(), connection.service, null,
null, null, null, 0, connection.addr, 0, null);
- connection.rpcServer.metrics.exception(SimpleRpcServer.REQUEST_TOO_BIG_EXCEPTION);
+ RequestTooBigException reqTooBigEx = new RequestTooBigException(requestTooBigMessage);
+ connection.rpcServer.metrics.exception(reqTooBigEx);
// Make sure the client recognizes the underlying exception
// Otherwise, throw a DoNotRetryIOException.
if (VersionInfoUtil.hasMinimumVersion(connection.connectionHeader.getVersionInfo(),
RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION)) {
- reqTooBig.setResponse(null, null,
- SimpleRpcServer.REQUEST_TOO_BIG_EXCEPTION, requestTooBigMessage);
+ reqTooBig.setResponse(null, null, reqTooBigEx, requestTooBigMessage);
} else {
- reqTooBig.setResponse(null, null, new DoNotRetryIOException(), requestTooBigMessage);
+ reqTooBig.setResponse(null, null, new DoNotRetryIOException(requestTooBigMessage),
+ requestTooBigMessage);
}
// To guarantee that the message is written and flushed before closing the channel,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 9a0c653..7fcd314 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -33,7 +33,6 @@ import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.LongAdder;
-
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CallQueueTooBigException;
@@ -41,7 +40,6 @@ import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
-import org.apache.hadoop.hbase.exceptions.RequestTooBigException;
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -176,8 +174,6 @@ public abstract class RpcServer implements RpcServerInterface,
protected HBaseRPCErrorHandler errorHandler = null;
public static final String MAX_REQUEST_SIZE = "hbase.ipc.max.request.size";
- protected static final RequestTooBigException REQUEST_TOO_BIG_EXCEPTION =
- new RequestTooBigException();
protected static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time";
protected static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java
index 01127cc..cfb2f26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java
@@ -28,18 +28,19 @@ import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
-
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.VersionInfoUtil;
import org.apache.hadoop.hbase.exceptions.RequestTooBigException;
import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup;
import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
@@ -211,14 +212,15 @@ class SimpleServerRpcConnection extends ServerRpcConnection {
SimpleServerCall reqTooBig = new SimpleServerCall(header.getCallId(), this.service, null,
null, null, null, this, 0, this.addr, System.currentTimeMillis(), 0,
this.rpcServer.bbAllocator, this.rpcServer.cellBlockBuilder, null, responder);
- this.rpcServer.metrics.exception(SimpleRpcServer.REQUEST_TOO_BIG_EXCEPTION);
+ RequestTooBigException reqTooBigEx = new RequestTooBigException(msg);
+ this.rpcServer.metrics.exception(reqTooBigEx);
// Make sure the client recognizes the underlying exception
// Otherwise, throw a DoNotRetryIOException.
if (VersionInfoUtil.hasMinimumVersion(connectionHeader.getVersionInfo(),
RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION)) {
- reqTooBig.setResponse(null, null, SimpleRpcServer.REQUEST_TOO_BIG_EXCEPTION, msg);
+ reqTooBig.setResponse(null, null, reqTooBigEx, msg);
} else {
- reqTooBig.setResponse(null, null, new DoNotRetryIOException(), msg);
+ reqTooBig.setResponse(null, null, new DoNotRetryIOException(msg), msg);
}
// In most cases we will write out the response directly. If not, it is still OK to just
// close the connection without writing out the reqTooBig response. Do not try to write
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 f81b3d4..c78bb00 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
@@ -18,7 +18,9 @@
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.assertThrows;
+
+import java.util.concurrent.ThreadLocalRandom;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
@@ -29,59 +31,52 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
-import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-@Category({MediumTests.class, ClientTests.class})
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+@Category({ MediumTests.class, ClientTests.class })
public class TestRequestTooBigException {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestRequestTooBigException.class);
+ HBaseClassTestRule.forClass(TestRequestTooBigException.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
- @Rule
- public TestName name = new TestName();
+ private static final TableName NAME = TableName.valueOf("request_too_big");
+
+ private static final byte[] FAMILY = Bytes.toBytes("family");
+
+ private static Table TABLE;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
- TEST_UTIL.getConfiguration().setInt(MAX_REQUEST_SIZE, 10000);
- TEST_UTIL.startMiniCluster();
+ TEST_UTIL.getConfiguration().setInt(MAX_REQUEST_SIZE, 10 * 1024);
+ TEST_UTIL.startMiniCluster(1);
+ TABLE = TEST_UTIL.createTable(NAME, FAMILY);
+ TEST_UTIL.waitTableAvailable(NAME);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
+ Closeables.close(TABLE, true);
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testHbasePutDeleteCell() throws Exception {
- final TableName tableName = TableName.valueOf(name.getMethodName());
- final byte[] family = Bytes.toBytes("cf");
- Table table = TEST_UTIL.createTable(tableName, family);
- TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
- try {
- byte[] value = new byte[2 * 2014 * 1024];
- 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);
- }
+ byte[] value = new byte[1024];
+ ThreadLocalRandom.current().nextBytes(value);
+ for (int m = 0; m < 100; m++) {
+ Put p = new Put(Bytes.toBytes("bigrow-" + m));
+ // max request is 10K, big request = 100 * 1K
+ for (int i = 0; i < 100; i++) {
+ p.addColumn(FAMILY, Bytes.toBytes("someQualifier" + i), value);
}
- } finally {
- table.close();
+ final Put finalPut = p;
+ assertThrows(RequestTooBigException.class, () -> TABLE.put(finalPut));
}
}
}
-
-