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:08:10 UTC

svn commit: r1451420 - in /hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver: HRegion.java wal/HLog.java

Author: larsh
Date: Thu Feb 28 23:08:09 2013
New Revision: 1451420

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

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1451420&r1=1451419&r2=1451420&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Feb 28 23:08:09 2013
@@ -364,6 +364,7 @@ public class HRegion implements HeapSize
   private HTableDescriptor htableDescriptor = null;
   private RegionSplitPolicy splitPolicy;
   private final OperationMetrics opMetrics;
+  private final boolean deferredLogSyncDisabled;
 
   /**
    * Should only be used for testing purposes
@@ -389,6 +390,7 @@ public class HRegion implements HeapSize
     this.maxBusyWaitDuration = 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
     this.busyWaitDuration = DEFAULT_BUSY_WAIT_DURATION;
     this.maxBusyWaitMultiplier = 2;
+    this.deferredLogSyncDisabled = false;
   }
 
   /**
@@ -457,7 +459,10 @@ public class HRegion implements HeapSize
     this.timestampSlop = conf.getLong(
         "hbase.hregion.keyvalue.timestamp.slop.millisecs",
         HConstants.LATEST_TIMESTAMP);
-
+    // 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
@@ -5644,7 +5649,7 @@ public class HRegion implements HeapSize
    */
   private void syncOrDefer(long txid) throws IOException {
     if (this.regionInfo.isMetaRegion() ||
-      !this.htableDescriptor.isDeferredLogFlush()) {
+      !this.htableDescriptor.isDeferredLogFlush() || this.deferredLogSyncDisabled) {
       this.log.sync(txid);
     }
   }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java?rev=1451420&r1=1451419&r2=1451420&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java Thu Feb 28 23:08:09 2013
@@ -244,7 +244,7 @@ public class HLog implements 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;
@@ -438,9 +438,16 @@ public class HLog implements Syncable {
     // handle the reflection necessary to call getNumCurrentReplicas()
     this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out);
 
-    logSyncerThread = new LogSyncer(this.optionalFlushInterval);
-    Threads.setDaemonThreadRunning(logSyncerThread.getThread(),
-        Thread.currentThread().getName() + ".logSyncer");
+    logSyncer = new LogSyncer(this.optionalFlushInterval);
+    // 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);
   }
   
@@ -985,12 +992,15 @@ public class HLog implements Syncable {
    * @throws IOException
    */
   public void close() throws IOException {
-    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);
+    // 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);
+      }
     }
 
     cacheFlushLock.lock();
@@ -1310,9 +1320,9 @@ public class HLog implements 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);
@@ -1323,7 +1333,7 @@ public class HLog implements Syncable {
           synchronized (flushLock) {
             // HBASE-4387, HBASE-5623, retry with updateLock held
             tempWriter = this.writer;
-            logSyncerThread.hlogFlush(tempWriter, pending);
+            logSyncer.hlogFlush(tempWriter, pending);
           }
         }
       }
@@ -1473,7 +1483,7 @@ public class HLog implements Syncable {
       // coprocessor hook:
       if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) {
         // write to our buffer for the Hlog file.
-        logSyncerThread.append(new HLog.Entry(logKey, logEdit));
+        logSyncer.append(new HLog.Entry(logKey, logEdit));
       }
       long took = System.currentTimeMillis() - now;
       coprocessorHost.postWALWrite(info, logKey, logEdit);
@@ -1602,7 +1612,7 @@ public class HLog implements Syncable {
         WALEdit edit = completeCacheFlushLogEdit();
         HLogKey key = makeKey(encodedRegionName, tableName, logSeqId,
             System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
-        logSyncerThread.append(new Entry(key, edit));
+        logSyncer.append(new Entry(key, edit));
         txid = this.unflushedEntries.incrementAndGet();
         writeTime.inc(System.currentTimeMillis() - now);
         long len = 0;