You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2013/03/01 00:24:36 UTC

svn commit: r1451427 - in /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver: HRegion.java wal/FSHLog.java

Author: larsh
Date: Thu Feb 28 23:24:35 2013
New Revision: 1451427

URL: http://svn.apache.org/r1451427
Log:
HBASE-7869 Provide way to not start LogSyncer thread (Anoop)

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1451427&r1=1451426&r2=1451427&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Feb 28 23:24:35 2013
@@ -380,6 +380,7 @@ public class HRegion implements HeapSize
 
   private final MetricsRegion metricsRegion;
   private final MetricsRegionWrapperImpl metricsRegionWrapper;
+  private final boolean deferredLogSyncDisabled;
 
   /**
    * HRegion constructor. This constructor should only be used for testing and
@@ -484,7 +485,10 @@ public class HRegion implements HeapSize
      */
     this.rowProcessorTimeout = conf.getLong(
         "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT);
-
+    // When hbase.regionserver.optionallogflushinterval <= 0 , deferred log sync is disabled.
+    this.deferredLogSyncDisabled = conf.getLong("hbase.regionserver.optionallogflushinterval",
+        1 * 1000) <= 0;
+    
     if (rsServices != null) {
       this.rsAccounting = this.rsServices.getRegionServerAccounting();
       // don't initialize coprocessors if not running within a regionserver
@@ -5381,7 +5385,7 @@ public class HRegion implements HeapSize
    */
   private void syncOrDefer(long txid) throws IOException {
     if (this.getRegionInfo().isMetaRegion() ||
-      !this.htableDescriptor.isDeferredLogFlush()) {
+      !this.htableDescriptor.isDeferredLogFlush() || this.deferredLogSyncDisabled) {
       this.log.sync(txid);
     }
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java?rev=1451427&r1=1451426&r2=1451427&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java Thu Feb 28 23:24:35 2013
@@ -220,7 +220,7 @@ class FSHLog implements HLog, Syncable {
   /**
    * Thread that handles optional sync'ing
    */
-  private final LogSyncer logSyncerThread;
+  private final LogSyncer logSyncer;
 
   /** Number of log close errors tolerated before we abort */
   private final int closeErrorsTolerated;
@@ -348,7 +348,7 @@ class FSHLog implements HLog, Syncable {
     this.closeErrorsTolerated = conf.getInt(
         "hbase.regionserver.logroll.errors.tolerated", 0);
     
-    this.logSyncerThread = new LogSyncer(this.optionalFlushInterval);
+    this.logSyncer = new LogSyncer(this.optionalFlushInterval);
     
     LOG.info("HLog configuration: blocksize=" +
       StringUtils.byteDesc(this.blocksize) +
@@ -378,8 +378,15 @@ class FSHLog implements HLog, Syncable {
     // handle the reflection necessary to call getNumCurrentReplicas()
     this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out);
 
-    Threads.setDaemonThreadRunning(logSyncerThread.getThread(),
-        Thread.currentThread().getName() + ".logSyncer");
+    // When optionalFlushInterval is set as 0, don't start a thread for deferred log sync.
+    if (this.optionalFlushInterval > 0) {
+      Threads.setDaemonThreadRunning(logSyncer.getThread(), Thread.currentThread().getName()
+          + ".logSyncer");
+    } else {
+      LOG.info("hbase.regionserver.optionallogflushinterval is set as "
+          + this.optionalFlushInterval + ". Deferred log syncing won't work. "
+          + "Any Mutation, marked to be deferred synced, will be flushed immediately.");
+    }
     coprocessorHost = new WALCoprocessorHost(this, conf);
 
     this.metrics = new MetricsWAL();
@@ -797,13 +804,16 @@ class FSHLog implements HLog, Syncable {
     if (this.closed) {
       return;
     }
-    try {
-      logSyncerThread.close();
-      // Make sure we synced everything
-      logSyncerThread.join(this.optionalFlushInterval*2);
-    } catch (InterruptedException e) {
-      LOG.error("Exception while waiting for syncer thread to die", e);
-      Thread.currentThread().interrupt();
+    // When optionalFlushInterval is 0, the logSyncer is not started as a Thread.
+    if (this.optionalFlushInterval > 0) {
+      try {
+        logSyncer.close();
+        // Make sure we synced everything
+        logSyncer.join(this.optionalFlushInterval * 2);
+      } catch (InterruptedException e) {
+        LOG.error("Exception while waiting for syncer thread to die", e);
+        Thread.currentThread().interrupt();
+      }
     }
     try {
       // Prevent all further flushing and rolling.
@@ -1088,9 +1098,9 @@ class FSHLog implements HLog, Syncable {
           return;
         }
         doneUpto = this.unflushedEntries.get();
-        pending = logSyncerThread.getPendingWrites();
+        pending = logSyncer.getPendingWrites();
         try {
-          logSyncerThread.hlogFlush(tempWriter, pending);
+          logSyncer.hlogFlush(tempWriter, pending);
         } catch(IOException io) {
           ioe = io;
           LOG.error("syncer encountered error, will retry. txid=" + txid, ioe);
@@ -1101,7 +1111,7 @@ class FSHLog implements HLog, Syncable {
           synchronized (flushLock) {
             // HBASE-4387, HBASE-5623, retry with updateLock held
             tempWriter = this.writer;
-            logSyncerThread.hlogFlush(tempWriter, pending);
+            logSyncer.hlogFlush(tempWriter, pending);
           }
         }          
       }
@@ -1256,7 +1266,7 @@ class FSHLog implements HLog, Syncable {
       // coprocessor hook:
       if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) {
         // write to our buffer for the Hlog file.
-        logSyncerThread.append(new FSHLog.Entry(logKey, logEdit));
+        logSyncer.append(new FSHLog.Entry(logKey, logEdit));
       }
       long took = EnvironmentEdgeManager.currentTimeMillis() - now;
       coprocessorHost.postWALWrite(info, logKey, logEdit);