HBASE-27756 Make request guardrail configs in RSRpcServices live updatable (#5152)
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
This commit is contained in:
parent
753eaefa29
commit
e77e3ffa8f
|
@ -348,7 +348,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
|
||||||
final InetSocketAddress isa;
|
final InetSocketAddress isa;
|
||||||
|
|
||||||
protected final HRegionServer regionServer;
|
protected final HRegionServer regionServer;
|
||||||
private final long maxScannerResultSize;
|
private volatile long maxScannerResultSize;
|
||||||
|
|
||||||
// The reference to the priority extraction function
|
// The reference to the priority extraction function
|
||||||
private final PriorityFunction priority;
|
private final PriorityFunction priority;
|
||||||
|
@ -377,12 +377,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
|
||||||
/**
|
/**
|
||||||
* Row size threshold for multi requests above which a warning is logged
|
* Row size threshold for multi requests above which a warning is logged
|
||||||
*/
|
*/
|
||||||
private final int rowSizeWarnThreshold;
|
private volatile int rowSizeWarnThreshold;
|
||||||
/*
|
/*
|
||||||
* Whether we should reject requests with very high no of rows i.e. beyond threshold defined by
|
* Whether we should reject requests with very high no of rows i.e. beyond threshold defined by
|
||||||
* rowSizeWarnThreshold
|
* rowSizeWarnThreshold
|
||||||
*/
|
*/
|
||||||
private final boolean rejectRowsWithSizeOverThreshold;
|
private volatile boolean rejectRowsWithSizeOverThreshold;
|
||||||
|
|
||||||
final AtomicBoolean clearCompactionQueues = new AtomicBoolean(false);
|
final AtomicBoolean clearCompactionQueues = new AtomicBoolean(false);
|
||||||
|
|
||||||
|
@ -1215,11 +1215,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
|
||||||
public RSRpcServices(final HRegionServer rs) throws IOException {
|
public RSRpcServices(final HRegionServer rs) throws IOException {
|
||||||
final Configuration conf = rs.getConfiguration();
|
final Configuration conf = rs.getConfiguration();
|
||||||
regionServer = rs;
|
regionServer = rs;
|
||||||
rowSizeWarnThreshold =
|
|
||||||
conf.getInt(HConstants.BATCH_ROWS_THRESHOLD_NAME, HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);
|
|
||||||
rejectRowsWithSizeOverThreshold =
|
|
||||||
conf.getBoolean(REJECT_BATCH_ROWS_OVER_THRESHOLD, DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD);
|
|
||||||
|
|
||||||
final RpcSchedulerFactory rpcSchedulerFactory;
|
final RpcSchedulerFactory rpcSchedulerFactory;
|
||||||
try {
|
try {
|
||||||
rpcSchedulerFactory = getRpcSchedulerFactoryClass().asSubclass(RpcSchedulerFactory.class)
|
rpcSchedulerFactory = getRpcSchedulerFactoryClass().asSubclass(RpcSchedulerFactory.class)
|
||||||
|
@ -1260,10 +1255,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
|
||||||
if (!(rs instanceof HMaster)) {
|
if (!(rs instanceof HMaster)) {
|
||||||
rpcServer.setNamedQueueRecorder(rs.getNamedQueueRecorder());
|
rpcServer.setNamedQueueRecorder(rs.getNamedQueueRecorder());
|
||||||
}
|
}
|
||||||
|
setReloadableGuardrails(conf);
|
||||||
scannerLeaseTimeoutPeriod = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
|
scannerLeaseTimeoutPeriod = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
|
||||||
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
|
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
|
||||||
maxScannerResultSize = conf.getLong(HConstants.HBASE_SERVER_SCANNER_MAX_RESULT_SIZE_KEY,
|
|
||||||
HConstants.DEFAULT_HBASE_SERVER_SCANNER_MAX_RESULT_SIZE);
|
|
||||||
rpcTimeout =
|
rpcTimeout =
|
||||||
conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
|
conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
|
||||||
minimumScanTimeLimitDelta = conf.getLong(REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA,
|
minimumScanTimeLimitDelta = conf.getLong(REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA,
|
||||||
|
@ -1310,6 +1304,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
|
||||||
public void onConfigurationChange(Configuration newConf) {
|
public void onConfigurationChange(Configuration newConf) {
|
||||||
if (rpcServer instanceof ConfigurationObserver) {
|
if (rpcServer instanceof ConfigurationObserver) {
|
||||||
((ConfigurationObserver) rpcServer).onConfigurationChange(newConf);
|
((ConfigurationObserver) rpcServer).onConfigurationChange(newConf);
|
||||||
|
setReloadableGuardrails(newConf);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -4111,4 +4106,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
|
||||||
.forEachRemaining(server -> builder.addNode(ProtobufUtil.toServerName(server)));
|
.forEachRemaining(server -> builder.addNode(ProtobufUtil.toServerName(server)));
|
||||||
return builder.build();
|
return builder.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void setReloadableGuardrails(Configuration conf) {
|
||||||
|
rowSizeWarnThreshold =
|
||||||
|
conf.getInt(HConstants.BATCH_ROWS_THRESHOLD_NAME, HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);
|
||||||
|
rejectRowsWithSizeOverThreshold =
|
||||||
|
conf.getBoolean(REJECT_BATCH_ROWS_OVER_THRESHOLD, DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD);
|
||||||
|
maxScannerResultSize = conf.getLong(HConstants.HBASE_SERVER_SCANNER_MAX_RESULT_SIZE_KEY,
|
||||||
|
HConstants.DEFAULT_HBASE_SERVER_SCANNER_MAX_RESULT_SIZE);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue