You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2009/12/16 00:38:39 UTC

svn commit: r891072 - in /hadoop/hbase/branches/0.20: CHANGES.txt src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Author: apurtell
Date: Tue Dec 15 23:38:38 2009
New Revision: 891072

URL: http://svn.apache.org/viewvc?rev=891072&view=rev
Log:
HBASE-2046 Remove CompactionLimitThread

Modified:
    hadoop/hbase/branches/0.20/CHANGES.txt
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Modified: hadoop/hbase/branches/0.20/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/CHANGES.txt?rev=891072&r1=891071&r2=891072&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.20/CHANGES.txt Tue Dec 15 23:38:38 2009
@@ -40,6 +40,7 @@
    HBASE-2027  HConnectionManager.HBASE_INSTANCES leaks TableServers
                (Dave Latham via Stack)
    HBASE-2031  When starting HQuorumPeer, try to match on more than 1 address
+   HBASE-2046  Remove CompactionLimitThread
 
 Release 0.20.2 - November 18th, 2009
   INCOMPATIBLE CHANGES

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=891072&r1=891071&r2=891072&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Tue Dec 15 23:38:38 2009
@@ -60,8 +60,6 @@
   
   private final HashSet<HRegion> regionsInQueue = new HashSet<HRegion>();
 
-  private volatile int limit = 1;
-
   /** @param server */
   public CompactSplitThread(HRegionServer server) {
     super();
@@ -81,18 +79,9 @@
         continue;
       }
     }
-    int count = 0;
     while (!this.server.isStopRequested()) {
       HRegion r = null;
       try {
-        if ((limit > 0) && (++count > limit)) {
-          try {
-            Thread.sleep(this.frequency);
-          } catch (InterruptedException ex) {
-            continue;
-          }
-          count = 0;
-        }
         r = compactionQueue.poll(this.frequency, TimeUnit.MILLISECONDS);
         if (r != null && !this.server.isStopRequested()) {
           synchronized (regionsInQueue) {
@@ -231,18 +220,6 @@
   }
 
   /**
-   * Sets the number of compactions allowed per cycle.
-   * @param limit the number of compactions allowed, or -1 to unlimit
-   */
-  void setLimit(int limit) {
-    this.limit = limit;
-  }
-
-  int getLimit() {
-    return this.limit;
-  }
-
-  /**
    * Only interrupt once it's done with a run through the work loop.
    */ 
   void interruptIfNecessary() {

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=891072&r1=891071&r2=891072&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Tue Dec 15 23:38:38 2009
@@ -207,9 +207,6 @@
   LogRoller hlogRoller;
   LogFlusher hlogFlusher;
   
-  // limit compactions while starting up
-  CompactionLimitThread compactionLimitThread;
-
   // flag set after we're done setting up server threads (used for testing)
   protected volatile boolean isOnline;
 
@@ -908,61 +905,6 @@
     return this.fsOk;
   }
 
-  /**
-   * Thread for toggling safemode after some configurable interval.
-   */
-  private class CompactionLimitThread extends Thread {
-    protected CompactionLimitThread() {}
-
-    @Override
-    public void run() {
-      // First wait until we exit safe mode
-      synchronized (safeMode) {
-        while(safeMode.get()) {
-          LOG.debug("Waiting to exit safe mode");
-          try {
-            safeMode.wait();
-          } catch (InterruptedException e) {
-            // ignore
-          }
-        }
-      }
-
-      // now that safemode is off, slowly increase the per-cycle compaction
-      // limit, finally setting it to unlimited (-1)
-
-      int compactionCheckInterval = 
-        conf.getInt("hbase.regionserver.thread.splitcompactcheckfrequency",
-            20 * 1000);
-      final int limitSteps[] = {
-        1, 1, 1, 1,
-        2, 2, 2, 2, 2, 2,
-        3, 3, 3, 3, 3, 3, 3, 3, 
-        4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
-        -1
-      };
-      for (int i = 0; i < limitSteps.length; i++) {
-        // Just log changes.
-        if (compactSplitThread.getLimit() != limitSteps[i] &&
-            LOG.isDebugEnabled()) {
-          LOG.debug("setting compaction limit to " + limitSteps[i]);
-        }
-        compactSplitThread.setLimit(limitSteps[i]);
-        try {
-          Thread.sleep(compactionCheckInterval);
-        } catch (InterruptedException ex) {
-          // unlimit compactions before exiting
-          compactSplitThread.setLimit(-1);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(this.getName() + " exiting on interrupt");
-          }
-          return;
-        }
-      }
-      LOG.info("compactions no longer limited");
-    }
-  }
-
   /*
    * Thread to shutdown the region server in an orderly manner.  This thread
    * is registered as a shutdown hook in the HRegionServer constructor and is
@@ -1226,17 +1168,6 @@
       } 
     }
 
-    // Set up the safe mode handler if safe mode has been configured.
-    if (!conf.getBoolean("hbase.regionserver.safemode", true)) {
-      safeMode.set(false);
-      compactSplitThread.setLimit(-1);
-      LOG.debug("skipping safe mode");
-    } else {
-      this.compactionLimitThread = new CompactionLimitThread();
-      Threads.setDaemonThreadRunning(this.compactionLimitThread, n + ".safeMode",
-        handler);
-    }
-
     // Start Server.  This service is like leases in that it internally runs
     // a thread.
     this.server.start();