You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ji...@apache.org on 2008/02/12 23:19:30 UTC

svn commit: r627152 - in /hadoop/hbase/trunk: CHANGES.txt conf/hbase-default.xml src/java/org/apache/hadoop/hbase/HRegionServer.java

Author: jimk
Date: Tue Feb 12 14:19:24 2008
New Revision: 627152

URL: http://svn.apache.org/viewvc?rev=627152&view=rev
Log:
HBASE-440   Add optional log roll interval so that log files are garbage collected

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/conf/hbase-default.xml
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=627152&r1=627151&r2=627152&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Tue Feb 12 14:19:24 2008
@@ -44,6 +44,8 @@
    HBASE-436   website: http://hadoop.apache.org/hbase
    HBASE-417   Factor TableOperation and subclasses into separate files from
                HMaster (Bryan Duxbury via Stack)
+   HBASE-440   Add optional log roll interval so that log files are garbage
+               collected
 
 
 Branch 0.1

Modified: hadoop/hbase/trunk/conf/hbase-default.xml
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/conf/hbase-default.xml?rev=627152&r1=627151&r2=627152&view=diff
==============================================================================
--- hadoop/hbase/trunk/conf/hbase-default.xml (original)
+++ hadoop/hbase/trunk/conf/hbase-default.xml Tue Feb 12 14:19:24 2008
@@ -156,6 +156,16 @@
     </description>
   </property>
   <property>
+    <name>hbase.regionserver.optionallogrollinterval</name>
+    <value>1800000</value>
+    <description>
+    Amount of time to wait since the last time a the region server's log was
+    rolled before invoking an optional log roll (An optional log roll is a
+    one in which the log does not contain hbase.regionserver.maxlogentries).
+    Default: 30 minutes (in miliseconds)
+    </description>
+  </property>
+  <property>
     <name>hbase.hregion.memcache.flush.size</name>
     <value>67108864</value>
     <description>

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java?rev=627152&r1=627151&r2=627152&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java Tue Feb 12 14:19:24 2008
@@ -508,12 +508,17 @@
   /** Runs periodically to determine if the HLog should be rolled */
   class LogRoller extends Thread implements LogRollListener {
     private final Integer rollLock = new Integer(0);
+    private final long optionalLogRollInterval;
+    private long lastLogRollTime;
     private volatile boolean rollLog;
     
     /** constructor */
     public LogRoller() {
       super();
+      this.optionalLogRollInterval = conf.getLong(
+          "hbase.regionserver.optionallogrollinterval", 30L * 60L * 1000L);
       this.rollLog = false;
+      lastLogRollTime = System.currentTimeMillis();
     }
  
     /** {@inheritDoc} */
@@ -521,12 +526,18 @@
     public void run() {
       while (!stopRequested.get()) {
         while (!rollLog && !stopRequested.get()) {
-          synchronized (rollLock) {
-            try {
-              rollLock.wait(threadWakeFrequency);
+          long now = System.currentTimeMillis();
+          if (this.lastLogRollTime + this.optionalLogRollInterval <= now) {
+            rollLog = true;
+            this.lastLogRollTime = now;
+          } else {
+            synchronized (rollLock) {
+              try {
+                rollLock.wait(threadWakeFrequency);
 
-            } catch (InterruptedException e) {
-              continue;
+              } catch (InterruptedException e) {
+                continue;
+              }
             }
           }
         }