You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by vj...@apache.org on 2020/09/16 10:24:54 UTC

[hbase] branch branch-2.2 updated: HBASE-25004 : Log RegionTooBusyException details (#2371)

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

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


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new 549bab7  HBASE-25004 : Log RegionTooBusyException details (#2371)
549bab7 is described below

commit 549bab70cb15c2816bb62a6fdbef59e8e3db4663
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Thu Sep 10 21:22:37 2020 +0530

    HBASE-25004 : Log RegionTooBusyException details (#2371)
    
    Signed-off-by: mnpoonia <ap...@salesforce.com>
    Signed-off-by: stack <st...@apache.org>
---
 .../apache/hadoop/hbase/regionserver/HRegion.java  | 31 ++++++++++++++--------
 1 file changed, 20 insertions(+), 11 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 5264801..97306d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -4455,12 +4455,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       requestFlush();
       // Don't print current limit because it will vary too much. The message is used as a key
       // over in RetriesExhaustedWithDetailsException processing.
-      throw new RegionTooBusyException("Over memstore limit=" +
-        org.apache.hadoop.hbase.procedure2.util.StringUtils.humanSize(this.blockingMemStoreSize) +
-        ", regionName=" +
-          (this.getRegionInfo() == null? "unknown": this.getRegionInfo().getEncodedName()) +
-          ", server=" + (this.getRegionServerServices() == null? "unknown":
-              this.getRegionServerServices().getServerName()));
+      final String regionName =
+        this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getEncodedName();
+      final String serverName = this.getRegionServerServices() == null ?
+        "unknown" : (this.getRegionServerServices().getServerName() == null ? "unknown" :
+          this.getRegionServerServices().getServerName().toString());
+      RegionTooBusyException rtbe = new RegionTooBusyException(
+        "Over memstore limit=" + org.apache.hadoop.hbase.procedure2.util.StringUtils
+          .humanSize(this.blockingMemStoreSize) + ", regionName=" + regionName + ", server="
+          + serverName);
+      LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe);
+      throw rtbe;
     }
   }
 
@@ -8565,11 +8570,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
         // Don't print millis. Message is used as a key over in
         // RetriesExhaustedWithDetailsException processing.
-        throw new RegionTooBusyException("Failed to obtain lock; regionName=" +
-            (this.getRegionInfo() == null? "unknown":
-                this.getRegionInfo().getRegionNameAsString()) +
-            ", server=" + (this.getRegionServerServices() == null? "unknown":
-                this.getRegionServerServices().getServerName()));
+        final String regionName =
+          this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
+        final String serverName = this.getRegionServerServices() == null ?
+          "unknown" : (this.getRegionServerServices().getServerName() == null ?
+            "unknown" : this.getRegionServerServices().getServerName().toString());
+        RegionTooBusyException rtbe = new RegionTooBusyException(
+          "Failed to obtain lock; regionName=" + regionName + ", server=" + serverName);
+        LOG.warn("Region is too busy to allow lock acquisition.", rtbe);
+        throw rtbe;
       }
     } catch (InterruptedException ie) {
       LOG.info("Interrupted while waiting for a lock in region {}", this);