hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject svn commit: r1528568 - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/master/SplitLogManager.java test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
Date Wed, 02 Oct 2013 17:21:48 GMT
Author: tedyu
Date: Wed Oct  2 17:21:47 2013
New Revision: 1528568

URL: http://svn.apache.org/r1528568
Log:
HBASE-9598 Non thread safe increment of task.unforcedResubmits in SplitLogManager#resubmit()


Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java?rev=1528568&r1=1528567&r2=1528568&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
Wed Oct  2 17:21:47 2013
@@ -33,6 +33,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
@@ -856,7 +857,7 @@ public class SplitLogManager extends Zoo
             " while the timeout is " + timeout);
         return false;
       }
-      if (task.unforcedResubmits >= resubmit_threshold) {
+      if (task.unforcedResubmits.get() >= resubmit_threshold) {
         if (!task.resubmitThresholdReached) {
           task.resubmitThresholdReached = true;
           SplitLogCounters.tot_mgr_resubmit_threshold_reached.incrementAndGet();
@@ -904,7 +905,7 @@ public class SplitLogManager extends Zoo
     }
     // don't count forced resubmits
     if (directive != FORCE) {
-      task.unforcedResubmits++;
+      task.unforcedResubmits.incrementAndGet();
     }
     task.setUnassigned();
     createRescanNode(Long.MAX_VALUE);
@@ -1281,7 +1282,7 @@ public class SplitLogManager extends Zoo
     volatile TaskBatch batch;
     volatile TerminationStatus status;
     volatile int incarnation;
-    volatile int unforcedResubmits;
+    volatile AtomicInteger unforcedResubmits = new AtomicInteger();
     volatile boolean resubmitThresholdReached;
 
     @Override
@@ -1291,7 +1292,7 @@ public class SplitLogManager extends Zoo
           " cur_worker_name = " + cur_worker_name +
           " status = " + status +
           " incarnation = " + incarnation +
-          " resubmits = " + unforcedResubmits +
+          " resubmits = " + unforcedResubmits.get() +
           " batch = " + batch);
     }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java?rev=1528568&r1=1528567&r2=1528568&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
(original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
Wed Oct  2 17:21:47 2013
@@ -255,7 +255,7 @@ public class TestSplitLogManager {
     LOG.debug("task = " + task);
     assertEquals(1L, tot_mgr_resubmit.get());
     assertEquals(1, task.incarnation);
-    assertEquals(0, task.unforcedResubmits);
+    assertEquals(0, task.unforcedResubmits.get());
     assertTrue(task.isOrphan());
     assertTrue(task.isUnassigned());
     assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);



Mime
View raw message