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 2011/05/20 00:00:27 UTC

svn commit: r1125140 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Author: stack
Date: Thu May 19 22:00:27 2011
New Revision: 1125140

URL: http://svn.apache.org/viewvc?rev=1125140&view=rev
Log:
HBASE-3826 Minor compaction needs to check if still over compactionThreshold after compacting

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1125140&r1=1125139&r2=1125140&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Thu May 19 22:00:27 2011
@@ -107,6 +107,8 @@ Release 0.91.0 - Unreleased
                jenkins and local
    HBASE-3897  Docs (notsoquick guide) suggest invalid XML (Philip Zeyliger)
    HBASE-3898  TestSplitTransactionOnCluster broke in TRUNK
+   HBASE-3826  Minor compaction needs to check if still over
+               compactionThreshold after compacting (Nicolas Spiegelberg)  
 
   IMPROVEMENTS
    HBASE-3290  Max Compaction Size (Nicolas Spiegelberg via Stack)  

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1125140&r1=1125139&r2=1125140&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu May 19 22:00:27 2011
@@ -133,6 +133,7 @@ import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.zookeeper.KeeperException;
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -232,9 +233,9 @@ public class HRegionServer implements HR
   MemStoreFlusher cacheFlusher;
 
   /*
-   * Check for major compactions.
+   * Check for compactions requests.
    */
-  Chore majorCompactionChecker;
+  Chore compactionChecker;
 
   // HLog and HLog roller. log is protected rather than private to avoid
   // eclipse warning when accessed by inner classes
@@ -565,11 +566,11 @@ public class HRegionServer implements HR
     // Compaction thread
     this.compactSplitThread = new CompactSplitThread(this);
 
-    // Background thread to check for major compactions; needed if region
+    // Background thread to check for compactions; needed if region
     // has not gotten updates in a while. Make it run at a lesser frequency.
     int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY +
       ".multiplier", 1000);
-    this.majorCompactionChecker = new MajorCompactionChecker(this,
+    this.compactionChecker = new CompactionChecker(this,
       this.threadWakeFrequency * multiplier, this);
 
     this.leases = new Leases((int) conf.getLong(
@@ -671,7 +672,8 @@ public class HRegionServer implements HR
     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
     if (this.hlogRoller != null) this.hlogRoller.interruptIfNecessary();
-    if (this.majorCompactionChecker != null) this.majorCompactionChecker.interrupt();
+    if (this.compactionChecker != null)
+      this.compactionChecker.interrupt();
 
     if (this.killed) {
       // Just skip out w/o closing regions.  Used when testing.
@@ -1030,17 +1032,16 @@ public class HRegionServer implements HR
   }
 
   /*
-   * Inner class that runs on a long period checking if regions need major
-   * compaction.
+   * Inner class that runs on a long period checking if regions need compaction.
    */
-  private static class MajorCompactionChecker extends Chore {
+  private static class CompactionChecker extends Chore {
     private final HRegionServer instance;
 
-    MajorCompactionChecker(final HRegionServer h, final int sleepTime,
+    CompactionChecker(final HRegionServer h, final int sleepTime,
         final Stoppable stopper) {
-      super("MajorCompactionChecker", sleepTime, h);
+      super("CompactionChecker", sleepTime, h);
       this.instance = h;
-      LOG.info("Runs every " + sleepTime + "ms");
+      LOG.info("Runs every " + StringUtils.formatTime(sleepTime));
     }
 
     @Override
@@ -1050,7 +1051,7 @@ public class HRegionServer implements HR
           continue;
         for (Store s : r.getStores().values()) {
           try {
-            if (s.isMajorCompaction()) {
+            if (s.isMajorCompaction() || s.needsCompaction()) {
               // Queue a compaction. Will recognize if major is needed.
               this.instance.compactSplitThread.requestCompaction(r, s,
                   getName() + " requests major compaction");
@@ -1247,8 +1248,8 @@ public class HRegionServer implements HR
     Threads.setDaemonThreadRunning(this.hlogRoller, n + ".logRoller", handler);
     Threads.setDaemonThreadRunning(this.cacheFlusher, n + ".cacheFlusher",
       handler);
-    Threads.setDaemonThreadRunning(this.majorCompactionChecker, n +
-      ".majorCompactionChecker", handler);
+    Threads.setDaemonThreadRunning(this.compactionChecker, n +
+      ".compactionChecker", handler);
 
     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
     // an unhandled exception, it will just exit.
@@ -1316,7 +1317,7 @@ public class HRegionServer implements HR
     // Verify that all threads are alive
     if (!(leases.isAlive()
         && cacheFlusher.isAlive() && hlogRoller.isAlive()
-        && this.majorCompactionChecker.isAlive())) {
+        && this.compactionChecker.isAlive())) {
       stop("One or more threads are no longer alive -- stop");
       return false;
     }
@@ -1430,7 +1431,7 @@ public class HRegionServer implements HR
    * have already been called.
    */
   protected void join() {
-    Threads.shutdown(this.majorCompactionChecker);
+    Threads.shutdown(this.compactionChecker);
     Threads.shutdown(this.cacheFlusher);
     Threads.shutdown(this.hlogRoller);
     if (this.compactSplitThread != null) {