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 2011/06/25 13:41:39 UTC

svn commit: r1139526 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java

Author: tedyu
Date: Sat Jun 25 11:41:39 2011
New Revision: 1139526

URL: http://svn.apache.org/viewvc?rev=1139526&view=rev
Log:
HBASE-4028  Hmaster crashes caused by splitting log.
   (gaojinchao via Ted Yu)


Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1139526&r1=1139525&r2=1139526&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Sat Jun 25 11:41:39 2011
@@ -135,6 +135,8 @@ Release 0.91.0 - Unreleased
                (Vandana Ayyalasomayajula via Ted Yu)
    HBASE-4029  Inappropriate checking of Logging Mode in HRegionServer
                (Akash Ashok via Ted Yu)
+   HBASE-4028  Hmaster crashes caused by splitting log.
+               (gaojinchao via Ted Yu)
 
   IMPROVEMENTS
    HBASE-3290  Max Compaction Size (Nicolas Spiegelberg via Stack)  

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java?rev=1139526&r1=1139525&r2=1139526&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java Sat Jun 25 11:41:39 2011
@@ -814,19 +814,20 @@ public class HLogSplitter {
       HLogKey key = entry.getKey();
 
       RegionEntryBuffer buffer;
+      long incrHeap;
       synchronized (this) {
         buffer = buffers.get(key.getEncodedRegionName());
         if (buffer == null) {
           buffer = new RegionEntryBuffer(key.getTablename(), key.getEncodedRegionName());
           buffers.put(key.getEncodedRegionName(), buffer);
         }
-        long incrHeap = buffer.appendEntry(entry);
-        totalBuffered += incrHeap;
+        incrHeap= buffer.appendEntry(entry);        
       }
 
       // If we crossed the chunk threshold, wait for more space to be available
       synchronized (dataAvailable) {
-        while (totalBuffered > maxHeapUsage && thrown == null) {
+        totalBuffered += incrHeap;
+        while (totalBuffered > maxHeapUsage && thrown.get() == null) {
           LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads...");
           dataAvailable.wait(3000);
         }