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/30 13:12:42 UTC

[hbase] branch branch-2 updated: HBASE-27756 Make request guardrail configs in RSRpcServices live updatable (#5152)

This is an automated email from the ASF dual-hosted git repository.

bbeaudreault pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new e77e3ffa8ff HBASE-27756 Make request guardrail configs in RSRpcServices live updatable (#5152)
e77e3ffa8ff is described below

commit e77e3ffa8ff9bfbe5cd967661cc8d904e79ed8cc
Author: Bryan Beaudreault <bb...@apache.org>
AuthorDate: Thu Mar 30 09:12:26 2023 -0400

    HBASE-27756 Make request guardrail configs in RSRpcServices live updatable (#5152)
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
---
 .../hadoop/hbase/regionserver/RSRpcServices.java   | 24 +++++++++++++---------
 1 file changed, 14 insertions(+), 10 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 f8dd7f80f41..834e3a6301a 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
@@ -348,7 +348,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
   final InetSocketAddress isa;
 
   protected final HRegionServer regionServer;
-  private final long maxScannerResultSize;
+  private volatile long maxScannerResultSize;
 
   // The reference to the priority extraction function
   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
    */
-  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);
 
@@ -1215,11 +1215,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
   public RSRpcServices(final HRegionServer rs) throws IOException {
     final Configuration conf = rs.getConfiguration();
     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;
     try {
       rpcSchedulerFactory = getRpcSchedulerFactoryClass().asSubclass(RpcSchedulerFactory.class)
@@ -1260,10 +1255,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
     if (!(rs instanceof HMaster)) {
       rpcServer.setNamedQueueRecorder(rs.getNamedQueueRecorder());
     }
+    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,
@@ -1310,6 +1304,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
   public void onConfigurationChange(Configuration newConf) {
     if (rpcServer instanceof ConfigurationObserver) {
       ((ConfigurationObserver) rpcServer).onConfigurationChange(newConf);
+      setReloadableGuardrails(newConf);
     }
   }
 
@@ -4111,4 +4106,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.Blockin
       .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);
+  }
 }