HBASE-25254 Rewrite TestMultiLogThreshold to remove the LogDelegate in RSRpcServices (#2631)

Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
Duo Zhang 2020-11-08 21:47:18 +08:00 committed by GitHub
parent 671129df56
commit c36ee4fac0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 83 additions and 80 deletions

View File

@ -107,8 +107,8 @@ import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.namequeues.NamedQueuePayload;
import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
import org.apache.hadoop.hbase.net.Address;
@ -140,7 +140,6 @@ import org.apache.hadoop.hbase.security.access.AccessChecker;
import org.apache.hadoop.hbase.security.access.NoopAccessChecker;
import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DNS;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -248,6 +247,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanReques
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
@ -1112,34 +1112,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
// Exposed for testing
interface LogDelegate {
void logBatchWarning(String firstRegionName, int sum, int rowSizeWarnThreshold);
}
private static LogDelegate DEFAULT_LOG_DELEGATE = new LogDelegate() {
@Override
public void logBatchWarning(String firstRegionName, int sum, int rowSizeWarnThreshold) {
if (LOG.isWarnEnabled()) {
LOG.warn("Large batch operation detected (greater than " + rowSizeWarnThreshold
+ ") (HBASE-18023)." + " Requested Number of Rows: " + sum + " Client: "
+ RpcServer.getRequestUserName().orElse(null) + "/"
+ RpcServer.getRemoteAddress().orElse(null)
+ " first region in multi=" + firstRegionName);
}
}
};
private final LogDelegate ld;
public RSRpcServices(final HRegionServer rs) throws IOException {
this(rs, DEFAULT_LOG_DELEGATE);
}
// Directly invoked only for testing
RSRpcServices(final HRegionServer rs, final LogDelegate ld) throws IOException {
public RSRpcServices(final HRegionServer rs) throws IOException {
final Configuration conf = rs.getConfiguration();
this.ld = ld;
regionServer = rs;
rowSizeWarnThreshold = conf.getInt(
HConstants.BATCH_ROWS_THRESHOLD_NAME, HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);
@ -2627,12 +2602,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
sum += regionAction.getActionCount();
}
if (sum > rowSizeWarnThreshold) {
ld.logBatchWarning(firstRegionName, sum, rowSizeWarnThreshold);
LOG.warn("Large batch operation detected (greater than " + rowSizeWarnThreshold +
") (HBASE-18023)." + " Requested Number of Rows: " + sum + " Client: " +
RpcServer.getRequestUserName().orElse(null) + "/" +
RpcServer.getRemoteAddress().orElse(null) + " first region in multi=" + firstRegionName);
if (rejectRowsWithSizeOverThreshold) {
throw new ServiceException(
"Rejecting large batch operation for current batch with firstRegionName: "
+ firstRegionName + " , Requested Number of Rows: " + sum + " , Size Threshold: "
+ rowSizeWarnThreshold);
"Rejecting large batch operation for current batch with firstRegionName: " +
firstRegionName + " , Requested Number of Rows: " + sum + " , Size Threshold: " +
rowSizeWarnThreshold);
}
}
}

View File

@ -17,6 +17,14 @@
*/
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.atLeastOnce;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.verify;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
@ -26,16 +34,20 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.log4j.Appender;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
@ -52,21 +64,23 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
* via "Multi" commands) so classified as MediumTests
*/
@RunWith(Parameterized.class)
@Category(LargeTests.class)
@Category(MediumTests.class)
public class TestMultiLogThreshold {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMultiLogThreshold.class);
private static RSRpcServices SERVICES;
private static HBaseTestingUtility TEST_UTIL;
private static Configuration CONF;
private static final TableName NAME = TableName.valueOf("tableName");
private static final byte[] TEST_FAM = Bytes.toBytes("fam");
private static RSRpcServices.LogDelegate LD;
private static HRegionServer RS;
private static int THRESHOLD;
private HBaseTestingUtility util;
private Configuration conf;
private int threshold;
private HRegionServer rs;
private RSRpcServices services;
private Appender appender;
@Parameterized.Parameter
public static boolean rejectLargeBatchOp;
@ -78,20 +92,22 @@ public class TestMultiLogThreshold {
@Before
public void setupTest() throws Exception {
final TableName tableName = TableName.valueOf("tableName");
TEST_UTIL = new HBaseTestingUtility();
CONF = TEST_UTIL.getConfiguration();
THRESHOLD = CONF.getInt(HConstants.BATCH_ROWS_THRESHOLD_NAME,
HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);
CONF.setBoolean("hbase.rpc.rows.size.threshold.reject", rejectLargeBatchOp);
TEST_UTIL.startMiniCluster();
TEST_UTIL.createTable(tableName, TEST_FAM);
RS = TEST_UTIL.getRSForFirstRegionInTable(tableName);
util = new HBaseTestingUtility();
conf = util.getConfiguration();
threshold =
conf.getInt(HConstants.BATCH_ROWS_THRESHOLD_NAME, HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);
conf.setBoolean("hbase.rpc.rows.size.threshold.reject", rejectLargeBatchOp);
util.startMiniCluster();
util.createTable(NAME, TEST_FAM);
rs = util.getRSForFirstRegionInTable(NAME);
appender = mock(Appender.class);
LogManager.getLogger(RSRpcServices.class).addAppender(appender);
}
@After
public void tearDown() throws Exception {
TEST_UTIL.shutdownMiniCluster();
LogManager.getLogger(RSRpcServices.class).removeAppender(appender);
util.shutdownMiniCluster();
}
private enum ActionType {
@ -128,38 +144,47 @@ public class TestMultiLogThreshold {
}
builder.addRegionAction(rab.build());
}
LD = Mockito.mock(RSRpcServices.LogDelegate.class);
SERVICES = new RSRpcServices(RS, LD);
SERVICES.multi(rpcc, builder.build());
services = new RSRpcServices(rs);
services.multi(rpcc, builder.build());
}
private void assertLogBatchWarnings(boolean expected) {
ArgumentCaptor<LoggingEvent> captor = ArgumentCaptor.forClass(LoggingEvent.class);
verify(appender, atLeastOnce()).doAppend(captor.capture());
boolean actual = false;
for (LoggingEvent event : captor.getAllValues()) {
if (event.getLevel() == Level.WARN &&
event.getRenderedMessage().contains("Large batch operation detected")) {
actual = true;
break;
}
}
reset(appender);
assertEquals(expected, actual);
}
@Test
public void testMultiLogThresholdRegionActions() throws ServiceException, IOException {
try {
sendMultiRequest(THRESHOLD + 1, ActionType.REGION_ACTIONS);
Assert.assertFalse(rejectLargeBatchOp);
sendMultiRequest(threshold + 1, ActionType.REGION_ACTIONS);
assertFalse(rejectLargeBatchOp);
} catch (ServiceException e) {
Assert.assertTrue(rejectLargeBatchOp);
assertTrue(rejectLargeBatchOp);
}
Mockito.verify(LD, Mockito.times(1))
.logBatchWarning(Mockito.anyString(), Mockito.anyInt(), Mockito.anyInt());
assertLogBatchWarnings(true);
sendMultiRequest(THRESHOLD, ActionType.REGION_ACTIONS);
Mockito.verify(LD, Mockito.never())
.logBatchWarning(Mockito.anyString(), Mockito.anyInt(), Mockito.anyInt());
sendMultiRequest(threshold, ActionType.REGION_ACTIONS);
assertLogBatchWarnings(false);
try {
sendMultiRequest(THRESHOLD + 1, ActionType.ACTIONS);
Assert.assertFalse(rejectLargeBatchOp);
sendMultiRequest(threshold + 1, ActionType.ACTIONS);
assertFalse(rejectLargeBatchOp);
} catch (ServiceException e) {
Assert.assertTrue(rejectLargeBatchOp);
assertTrue(rejectLargeBatchOp);
}
Mockito.verify(LD, Mockito.times(1))
.logBatchWarning(Mockito.anyString(), Mockito.anyInt(), Mockito.anyInt());
assertLogBatchWarnings(true);
sendMultiRequest(THRESHOLD, ActionType.ACTIONS);
Mockito.verify(LD, Mockito.never())
.logBatchWarning(Mockito.anyString(), Mockito.anyInt(), Mockito.anyInt());
sendMultiRequest(threshold, ActionType.ACTIONS);
assertLogBatchWarnings(false);
}
}