You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2013/02/01 22:13:14 UTC

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

Author: tedyu
Date: Fri Feb  1 21:13:14 2013
New Revision: 1441631

URL: http://svn.apache.org/viewvc?rev=1441631&view=rev
Log:
HBASE-7728 deadlock occurs between hlog roller and hlog syncer (Ted Yu)


Modified:
    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/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=1441631&r1=1441630&r2=1441631&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 Fri Feb  1 21:13:14 2013
@@ -827,6 +827,7 @@ class FSHLog implements HLog, Syncable {
       }
       if (this.writer != null) {
         this.writer.close();
+        this.writer = null;
       }
     }
   }
@@ -1081,35 +1082,43 @@ class FSHLog implements HLog, Syncable {
       // issue the sync to HDFS. If sync is successful, then update
       // syncedTillHere to indicate that transactions till this
       // number has been successfully synced.
+      IOException ioe = null;
+      List<Entry> pending = null;
       synchronized (flushLock) {
         if (txid <= this.syncedTillHere) {
           return;
         }
         doneUpto = this.unflushedEntries.get();
-        List<Entry> pending = logSyncerThread.getPendingWrites();
+        pending = logSyncerThread.getPendingWrites();
         try {
           logSyncerThread.hlogFlush(tempWriter, pending);
         } catch(IOException io) {
-          synchronized (this.updateLock) {
+          ioe = io;
+          LOG.error("syncer encountered error, will retry. txid=" + txid, ioe);
+        }
+      }
+      if (ioe != null && pending != null) {
+        synchronized (this.updateLock) {
+          synchronized (flushLock) {
             // HBASE-4387, HBASE-5623, retry with updateLock held
             tempWriter = this.writer;
             logSyncerThread.hlogFlush(tempWriter, pending);
           }
-        }
+        }          
       }
       // another thread might have sync'ed avoid double-sync'ing
       if (txid <= this.syncedTillHere) {
         return;
       }
       try {
-        tempWriter.sync();
+        if (tempWriter != null) tempWriter.sync();
       } catch(IOException ex) {
         synchronized (this.updateLock) {
           // HBASE-4387, HBASE-5623, retry with updateLock held
           // TODO: we don't actually need to do it for concurrent close - what is the point
           //       of syncing new unrelated writer? Keep behavior for now.
           tempWriter = this.writer;
-          tempWriter.sync();
+          if (tempWriter != null) tempWriter.sync();
         }
       }
       this.syncedTillHere = Math.max(this.syncedTillHere, doneUpto);