You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 04:21:41 UTC

svn commit: r1181573 - /hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java

Author: nspiegelberg
Date: Tue Oct 11 02:21:40 2011
New Revision: 1181573

URL: http://svn.apache.org/viewvc?rev=1181573&view=rev
Log:
HLog.closed should be checked inside of updateLock

Summary:
port D270734

Test Plan:
none

Reviewed By: kannan
Reviewers: kannan, jgray
CC: hbase@lists, kannan
Differential Revision: 271019

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

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java?rev=1181573&r1=1181572&r2=1181573&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java Tue Oct 11 02:21:40 2011
@@ -195,7 +195,7 @@ public class HLog implements Syncable {
   private final ConcurrentSkipListMap<byte [], Long> lastSeqWritten =
     new ConcurrentSkipListMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
 
-  private volatile boolean closed = false;
+  private boolean closed = false;
 
   private final AtomicLong logSeqNum = new AtomicLong(0);
 
@@ -467,10 +467,10 @@ public class HLog implements Syncable {
     byte [][] regionsToFlush = null;
     this.cacheFlushLock.lock();
     try {
-      if (closed) {
-        return regionsToFlush;
-      }
       synchronized (updateLock) {
+        if (closed) {
+          return regionsToFlush;
+        }
         // Clean up current writer.
         Path oldFile = cleanupCurrentWriter(this.filenum);
         this.filenum = System.currentTimeMillis();
@@ -806,11 +806,11 @@ public class HLog implements Syncable {
    */
   public void append(HRegionInfo regionInfo, HLogKey logKey, WALEdit logEdit)
   throws IOException {
-    if (this.closed) {
-      throw new IOException("Cannot append; log is closed");
-    }
     byte [] regionName = regionInfo.getRegionName();
     synchronized (updateLock) {
+      if (this.closed) {
+        throw new IOException("Cannot append; log is closed");
+      }
       long seqNum = obtainSeqNum();
       logKey.setLogSeqNum(seqNum);
       // The 'lastSeqWritten' map holds the sequence number of the oldest
@@ -857,12 +857,12 @@ public class HLog implements Syncable {
     if (edits.isEmpty()) return;
 
     byte[] regionName = info.getRegionName();
-    if (this.closed) {
-      throw new IOException("Cannot append; log is closed");
-    }
     long txid = 0;
     long start = System.currentTimeMillis();
     synchronized (this.updateLock) {
+      if (this.closed) {
+        throw new IOException("Cannot append; log is closed");
+      }
       long seqNum = obtainSeqNum();
       // The 'lastSeqWritten' map holds the sequence number of the oldest
       // write for each region (i.e. the first edit added to the particular
@@ -1174,10 +1174,10 @@ public class HLog implements Syncable {
     final boolean isMetaRegion)
   throws IOException {
     try {
-      if (this.closed) {
-        return;
-      }
       synchronized (updateLock) {
+        if (this.closed) {
+          return;
+        }
         long now = System.currentTimeMillis();
         WALEdit edit = completeCacheFlushLogEdit();
         HLogKey key = makeKey(regionName, tableName, logSeqId,