You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2019/10/30 18:29:06 UTC

[hbase] branch branch-2 updated: HBASE-20827 Use backoff on CallQueueTooBigException when reporting region state transition

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

elserj 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 d5f6329  HBASE-20827 Use backoff on CallQueueTooBigException when reporting region state transition
d5f6329 is described below

commit d5f6329facf6cc063cea05cd075e5981c9cc000f
Author: Ankit Singhal <an...@apache.org>
AuthorDate: Wed Oct 30 11:29:05 2019 -0400

    HBASE-20827 Use backoff on CallQueueTooBigException when reporting region state transition
    
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../java/org/apache/hadoop/hbase/regionserver/HRegionServer.java    | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index f1df341..79985ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
+import org.apache.hadoop.hbase.CallQueueTooBigException;
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ClockOutOfSyncException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
@@ -2336,8 +2337,9 @@ public class HRegionServer extends HasThread implements
         return true;
       } catch (ServiceException se) {
         IOException ioe = ProtobufUtil.getRemoteException(se);
-        boolean pause = ioe instanceof ServerNotRunningYetException ||
-            ioe instanceof PleaseHoldException;
+        boolean pause =
+            ioe instanceof ServerNotRunningYetException || ioe instanceof PleaseHoldException
+                || ioe instanceof CallQueueTooBigException;
         if (pause) {
           // Do backoff else we flood the Master with requests.
           pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);