You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bb...@apache.org on 2023/03/29 20:20:39 UTC
[hbase] branch master updated: HBASE-27756 Make request guardrail configs in RSRpcServices live updatable (#5141)
This is an automated email from the ASF dual-hosted git repository.
bbeaudreault pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new e5620e26a27 HBASE-27756 Make request guardrail configs in RSRpcServices live updatable (#5141)
e5620e26a27 is described below
commit e5620e26a275052151865f30956aa33393542a4a
Author: Bryan Beaudreault <bb...@apache.org>
AuthorDate: Wed Mar 29 16:20:29 2023 -0400
HBASE-27756 Make request guardrail configs in RSRpcServices live updatable (#5141)
Signed-off-by: Nick Dimiduk <nd...@apache.org>
---
.../hadoop/hbase/regionserver/RSRpcServices.java | 28 +++++++++++++++-------
1 file changed, 19 insertions(+), 9 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 896b8e7180a..87879d638de 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -297,7 +297,7 @@ public class RSRpcServices extends HBaseRpcServicesBase<HRegionServer>
// Request counter for rpc mutate
final LongAdder rpcMutateRequestCount = new LongAdder();
- private final long maxScannerResultSize;
+ private volatile long maxScannerResultSize;
private ScannerIdGenerator scannerIdGenerator;
private final ConcurrentMap<String, RegionScannerHolder> scanners = new ConcurrentHashMap<>();
@@ -323,12 +323,12 @@ public class RSRpcServices extends HBaseRpcServicesBase<HRegionServer>
/**
* 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
* rowSizeWarnThreshold
*/
- private final boolean rejectRowsWithSizeOverThreshold;
+ private volatile boolean rejectRowsWithSizeOverThreshold;
final AtomicBoolean clearCompactionQueues = new AtomicBoolean(false);
@@ -1162,14 +1162,9 @@ public class RSRpcServices extends HBaseRpcServicesBase<HRegionServer>
public RSRpcServices(final HRegionServer rs) throws IOException {
super(rs, rs.getProcessName());
final Configuration conf = rs.getConfiguration();
- 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);
+ setReloadableGuardrails(conf);
scannerLeaseTimeoutPeriod = conf.getInt(HConstants.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 =
conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
minimumScanTimeLimitDelta = conf.getLong(REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA,
@@ -3923,4 +3918,19 @@ public class RSRpcServices extends HBaseRpcServicesBase<HRegionServer>
.forEachRemaining(server -> builder.addNode(ProtobufUtil.toServerName(server)));
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);
+ }
+
+ @Override
+ public void onConfigurationChange(Configuration conf) {
+ super.onConfigurationChange(conf);
+ setReloadableGuardrails(conf);
+ }
}