You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/04/09 20:18:31 UTC

svn commit: r1586066 - /hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java

Author: liyin
Date: Wed Apr  9 18:18:31 2014
New Revision: 1586066

URL: http://svn.apache.org/r1586066
Log:
[master] Make region readonly after the first flush completes during region close

Author: fan

Summary:
We need to use atomic booleans to disable mutations before the second flush starts. Otherwise IPC handler threads which are trying to mutate the region could be blocked on splitsAndClosesLock until the region is closed.
This diff gives IPC handler threads greater chance to return NSRE to clients immediately.

Test Plan: TestHRegionClose

Reviewers: liyintang, aaiyer

Reviewed By: liyintang

CC: hbase-dev@

Differential Revision: https://phabricator.fb.com/D1265645

Task ID: 4099120

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1586066&r1=1586065&r2=1586066&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Wed Apr  9 18:18:31 2014
@@ -886,7 +886,8 @@ public class HRegion implements HeapSize
         } catch (IOException ioe) {
           // Failed to flush the region but probably it is still able to serve request,
           // so re-enable writes to it.
-          status.setStatus("Failed to flush the region, putting it online again");
+          LOG.warn("Not able to flush " + this.getRegionNameAsString() +
+              " . Putting it online again");
           synchronized (writestate) {
             writestate.writesEnabled = true;
           }
@@ -895,6 +896,10 @@ public class HRegion implements HeapSize
       }
       newScannerLock.writeLock().lock();
       this.closing.set(true);
+      synchronized (writestate) {
+        writestate.readOnly = true;
+      }
+      LOG.debug("Region " + this.getRegionNameAsString() + " is readonly now");
       status.setStatus("Disabling writes for close");
       try {
         splitsAndClosesLock.writeLock().lock();
@@ -912,9 +917,11 @@ public class HRegion implements HeapSize
             try {
               internalFlushcache(status);
             } catch (IOException ioe) {
-              status.setStatus("Failed to flush the region, putting it online again");
+              LOG.warn("Not able to flush " + this.getRegionNameAsString() +
+                  " . Putting it online again");
               synchronized (writestate) {
                 writestate.writesEnabled = true;
+                writestate.readOnly = false;
               }
               this.closing.set(false);
               throw ioe;