Index: src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (revision 1234360) +++ src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (working copy) @@ -958,6 +958,7 @@ public void close() throws IOException { try { logSyncerThread.interrupt(); + logSyncerThread.close(); // Make sure we synced everything logSyncerThread.join(this.optionalFlushInterval*2); } catch (InterruptedException e) { @@ -1169,6 +1170,8 @@ class LogSyncer extends HasThread { private final long optionalFlushInterval; + + private boolean closeLogSyncer = false; // List of pending writes to the HLog. There corresponds to transactions // that have not yet returned to the client. We keep them cached here @@ -1187,7 +1190,7 @@ try { // awaiting with a timeout doesn't always // throw exceptions on interrupt - while(!this.isInterrupted()) { + while(!this.isInterrupted() && !closeLogSyncer) { try { if (unflushedEntries.get() <= syncedTillHere) { @@ -1232,6 +1235,10 @@ writer.append(e); } } + + void close(){ + closeLogSyncer = true; + } } // sync all known transactions