HBASE-26179 TestRequestTooBigException spends too much time to finish (#3571)

Signed-off-by: Yulin Niu <niuyulin@apache.org>
This commit is contained in:
Duo Zhang 2021-08-07 23:01:55 +08:00 committed by GitHub
parent c9383f2c89
commit a79a9cc53e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 40 additions and 45 deletions

View File

@ -130,16 +130,17 @@ public class NettyRpcFrameDecoder extends ByteToMessageDecoder {
NettyServerCall reqTooBig = connection.createCall(header.getCallId(), connection.service, null, NettyServerCall reqTooBig = connection.createCall(header.getCallId(), connection.service, null,
null, null, null, 0, connection.addr, 0, 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 // Make sure the client recognizes the underlying exception
// Otherwise, throw a DoNotRetryIOException. // Otherwise, throw a DoNotRetryIOException.
if (VersionInfoUtil.hasMinimumVersion(connection.connectionHeader.getVersionInfo(), if (VersionInfoUtil.hasMinimumVersion(connection.connectionHeader.getVersionInfo(),
RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION)) { RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION)) {
reqTooBig.setResponse(null, null, reqTooBig.setResponse(null, null, reqTooBigEx, requestTooBigMessage);
SimpleRpcServer.REQUEST_TOO_BIG_EXCEPTION, requestTooBigMessage);
} else { } 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, // To guarantee that the message is written and flushed before closing the channel,

View File

@ -41,13 +41,12 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.conf.ConfigurationObserver; 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.io.ByteBuffAllocator;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder; import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.security.HBasePolicyProvider; import org.apache.hadoop.hbase.security.HBasePolicyProvider;
import org.apache.hadoop.hbase.security.SaslUtil; import org.apache.hadoop.hbase.security.SaslUtil;
import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
@ -177,8 +176,6 @@ public abstract class RpcServer implements RpcServerInterface,
protected HBaseRPCErrorHandler errorHandler = null; protected HBaseRPCErrorHandler errorHandler = null;
public static final String MAX_REQUEST_SIZE = "hbase.ipc.max.request.size"; 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_TIME = "hbase.ipc.warn.response.time";
protected static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size"; protected static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";

View File

@ -28,19 +28,20 @@ import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.VersionInfoUtil; import org.apache.hadoop.hbase.client.VersionInfoUtil;
import org.apache.hadoop.hbase.exceptions.RequestTooBigException; import org.apache.hadoop.hbase.exceptions.RequestTooBigException;
import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup; import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; 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.CodedInputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
import org.apache.hbase.thirdparty.com.google.protobuf.Message; import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
@ -212,14 +213,15 @@ class SimpleServerRpcConnection extends ServerRpcConnection {
SimpleServerCall reqTooBig = new SimpleServerCall(header.getCallId(), this.service, null, SimpleServerCall reqTooBig = new SimpleServerCall(header.getCallId(), this.service, null,
null, null, null, this, 0, this.addr, EnvironmentEdgeManager.currentTime(), 0, null, null, null, this, 0, this.addr, EnvironmentEdgeManager.currentTime(), 0,
this.rpcServer.bbAllocator, this.rpcServer.cellBlockBuilder, null, responder); 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 // Make sure the client recognizes the underlying exception
// Otherwise, throw a DoNotRetryIOException. // Otherwise, throw a DoNotRetryIOException.
if (VersionInfoUtil.hasMinimumVersion(connectionHeader.getVersionInfo(), if (VersionInfoUtil.hasMinimumVersion(connectionHeader.getVersionInfo(),
RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION)) { RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION)) {
reqTooBig.setResponse(null, null, SimpleRpcServer.REQUEST_TOO_BIG_EXCEPTION, msg); reqTooBig.setResponse(null, null, reqTooBigEx, msg);
} else { } 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 // 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 // close the connection without writing out the reqTooBig response. Do not try to write

View File

@ -18,7 +18,9 @@
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.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.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;
@ -29,59 +31,52 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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 { public class TestRequestTooBigException {
@ClassRule @ClassRule
public static final HBaseClassTestRule CLASS_RULE = public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestRequestTooBigException.class); HBaseClassTestRule.forClass(TestRequestTooBigException.class);
private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
@Rule private static final TableName NAME = TableName.valueOf("request_too_big");
public TestName name = new TestName();
private static final byte[] FAMILY = Bytes.toBytes("family");
private static Table TABLE;
@BeforeClass @BeforeClass
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().setInt(MAX_REQUEST_SIZE, 10000); TEST_UTIL.getConfiguration().setInt(MAX_REQUEST_SIZE, 10 * 1024);
TEST_UTIL.startMiniCluster(); TEST_UTIL.startMiniCluster(1);
TABLE = TEST_UTIL.createTable(NAME, FAMILY);
TEST_UTIL.waitTableAvailable(NAME);
} }
@AfterClass @AfterClass
public static void tearDownAfterClass() throws Exception { public static void tearDownAfterClass() throws Exception {
Closeables.close(TABLE, true);
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }
@Test @Test
public void testHbasePutDeleteCell() throws Exception { public void testHbasePutDeleteCell() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName()); byte[] value = new byte[1024];
final byte[] family = Bytes.toBytes("cf"); ThreadLocalRandom.current().nextBytes(value);
Table table = TEST_UTIL.createTable(tableName, family); for (int m = 0; m < 100; m++) {
TEST_UTIL.waitTableAvailable(tableName.getName(), 5000); Put p = new Put(Bytes.toBytes("bigrow-" + m));
try { // max request is 10K, big request = 100 * 1K
byte[] value = new byte[2 * 2014 * 1024]; for (int i = 0; i < 100; i++) {
for (int m = 0; m < 10000; m++) { p.addColumn(FAMILY, Bytes.toBytes("someQualifier" + i), value);
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 { final Put finalPut = p;
table.close(); assertThrows(RequestTooBigException.class, () -> TABLE.put(finalPut));
} }
} }
} }