You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/07/16 23:34:28 UTC

svn commit: r794872 - in /hadoop/hbase/branches/0.19: CHANGES.txt src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java

Author: stack
Date: Thu Jul 16 21:34:27 2009
New Revision: 794872

URL: http://svn.apache.org/viewvc?rev=794872&view=rev
Log:
HBASE-1664 Disable 1058 on catalog tables

Modified:
    hadoop/hbase/branches/0.19/CHANGES.txt
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java

Modified: hadoop/hbase/branches/0.19/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/CHANGES.txt?rev=794872&r1=794871&r2=794872&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.19/CHANGES.txt Thu Jul 16 21:34:27 2009
@@ -22,6 +22,7 @@
    HBASE-698   HLog recovery is not performed after master failure
    HBASE-1662  Tool to run major compaction on catalog regions when hbase
                is shutdown
+   HBASE-1664  Disable 1058 on catalog tables
 
 Release 0.19.3 - May 27th, 2009
   BUG FIXES

Modified: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java?rev=794872&r1=794871&r2=794872&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java (original)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java Thu Jul 16 21:34:27 2009
@@ -221,41 +221,7 @@
    * not flushed.
    */
   private boolean flushRegion(HRegion region, boolean removeFromQueue) {
-    // Wait until it is safe to flush
-    int count = 0;
-    boolean triggered = false;
-    boolean finished = false;
-    while (count++ < (blockingWaitTime / 500)) {
-      for (HStore hstore: region.stores.values()) {
-        if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) {
-          // always request a compaction
-          server.compactSplitThread.compactionRequested(region, getName());
-          // only log once
-          if (!triggered) {
-            LOG.info("Too many store files for region " + region + ": " +
-              hstore.getStorefilesCount() + ", requesting compaction and " +
-              "waiting");
-            triggered = true;
-          }
-          // pending compaction, not finished
-          finished = false;
-          try {
-            Thread.sleep(500);
-          } catch (InterruptedException e) {
-            // ignore
-          }
-        }
-      }
-      if(triggered && finished) {
-        LOG.info("Compaction has completed, we waited " + (count * 500) + "ms, "
-            + "finishing flush of region " + region);
-        break;
-      }
-    }
-    if(triggered && !finished) {
-      LOG.warn("Tried to hold up flushing for compactions of region " + region +
-          " but have waited longer than " + blockingWaitTime + "ms, continuing");
-    }
+    checkStoreFileCount(region);
     synchronized (regionsInQueue) {
       // See comment above for removeFromQueue on why we do not
       // take the region out of the set. If removeFromQueue is true, remove it
@@ -296,7 +262,53 @@
 
     return true;
   }
-  
+
+  /*
+   * If too many store files already, schedule a compaction and pause a while
+   * before going on with compaction.
+   * @param region Region to check.
+   */
+  private void checkStoreFileCount(final HRegion region) {
+    // If catalog region, do not ever hold up writes (isMetaRegion returns
+    // true if ROOT or META region).
+    if (region.getRegionInfo().isMetaRegion()) return;
+
+    int count = 0;
+    boolean triggered = false;
+    boolean finished = false;
+    while (count++ < (blockingWaitTime / 500)) {
+      finished = true;
+      for (HStore hstore: region.stores.values()) {
+        if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) {
+          // only log once
+          if (!triggered) {
+            LOG.info("Too many store files for region " + region + ": " +
+              hstore.getStorefilesCount() + ", requesting compaction and " +
+              "waiting");
+            this.server.compactSplitThread.compactionRequested(region, getName());
+            triggered = true;
+          }
+          // pending compaction, not finished
+          finished = false;
+          try {
+            Thread.sleep(500);
+          } catch (InterruptedException e) {
+            // ignore
+          }
+        }
+      }
+      if (triggered && finished) {
+        LOG.info("Compaction has completed, we waited " + (count * 500) + "ms, "
+            + "finishing flush of region " + region);
+        break;
+      }
+    }
+    if (triggered && !finished) {
+      LOG.warn("Tried to hold up flushing for compactions of region " + region +
+          " but have waited longer than " + blockingWaitTime + "ms, continuing");
+    }
+  }
+
   /**
    * Check if the regionserver's memcache memory usage is greater than the 
    * limit. If so, flush regions with the biggest memcaches until we're down