HBASE-9598 Non thread safe increment of task.unforcedResubmits in SplitLogManager#resubmit()
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1528568 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
3d0c0cb394
commit
03adb3aae4
|
@ -33,6 +33,7 @@ import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.locks.ReentrantLock;
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
|
@ -856,7 +857,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
||||||
" while the timeout is " + timeout);
|
" while the timeout is " + timeout);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
if (task.unforcedResubmits >= resubmit_threshold) {
|
if (task.unforcedResubmits.get() >= resubmit_threshold) {
|
||||||
if (!task.resubmitThresholdReached) {
|
if (!task.resubmitThresholdReached) {
|
||||||
task.resubmitThresholdReached = true;
|
task.resubmitThresholdReached = true;
|
||||||
SplitLogCounters.tot_mgr_resubmit_threshold_reached.incrementAndGet();
|
SplitLogCounters.tot_mgr_resubmit_threshold_reached.incrementAndGet();
|
||||||
|
@ -904,7 +905,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
||||||
}
|
}
|
||||||
// don't count forced resubmits
|
// don't count forced resubmits
|
||||||
if (directive != FORCE) {
|
if (directive != FORCE) {
|
||||||
task.unforcedResubmits++;
|
task.unforcedResubmits.incrementAndGet();
|
||||||
}
|
}
|
||||||
task.setUnassigned();
|
task.setUnassigned();
|
||||||
createRescanNode(Long.MAX_VALUE);
|
createRescanNode(Long.MAX_VALUE);
|
||||||
|
@ -1281,7 +1282,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
||||||
volatile TaskBatch batch;
|
volatile TaskBatch batch;
|
||||||
volatile TerminationStatus status;
|
volatile TerminationStatus status;
|
||||||
volatile int incarnation;
|
volatile int incarnation;
|
||||||
volatile int unforcedResubmits;
|
volatile AtomicInteger unforcedResubmits = new AtomicInteger();
|
||||||
volatile boolean resubmitThresholdReached;
|
volatile boolean resubmitThresholdReached;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1291,7 +1292,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
||||||
" cur_worker_name = " + cur_worker_name +
|
" cur_worker_name = " + cur_worker_name +
|
||||||
" status = " + status +
|
" status = " + status +
|
||||||
" incarnation = " + incarnation +
|
" incarnation = " + incarnation +
|
||||||
" resubmits = " + unforcedResubmits +
|
" resubmits = " + unforcedResubmits.get() +
|
||||||
" batch = " + batch);
|
" batch = " + batch);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -255,7 +255,7 @@ public class TestSplitLogManager {
|
||||||
LOG.debug("task = " + task);
|
LOG.debug("task = " + task);
|
||||||
assertEquals(1L, tot_mgr_resubmit.get());
|
assertEquals(1L, tot_mgr_resubmit.get());
|
||||||
assertEquals(1, task.incarnation);
|
assertEquals(1, task.incarnation);
|
||||||
assertEquals(0, task.unforcedResubmits);
|
assertEquals(0, task.unforcedResubmits.get());
|
||||||
assertTrue(task.isOrphan());
|
assertTrue(task.isOrphan());
|
||||||
assertTrue(task.isUnassigned());
|
assertTrue(task.isUnassigned());
|
||||||
assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);
|
assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);
|
||||||
|
|
Loading…
Reference in New Issue