HBASE-8954 TestSplitLogWorker#testPreemptTask failed

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1503512 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
jxiang 2013-07-15 23:10:33 +00:00
parent 83f51a4d6a
commit 8eb2f3709a
3 changed files with 5 additions and 0 deletions

View File

@ -82,6 +82,7 @@ public class SplitLogCounters {
public final static AtomicLong tot_wkr_preempt_task = new AtomicLong(0);
public final static AtomicLong tot_wkr_task_heartbeat_failed = new AtomicLong(0);
public final static AtomicLong tot_wkr_final_transition_failed = new AtomicLong(0);
public final static AtomicLong tot_wkr_task_grabing = new AtomicLong(0);
public static void resetCounters() throws Exception {
Class<?> cl = (new SplitLogCounters()).getClass();

View File

@ -240,6 +240,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
return;
}
}
SplitLogCounters.tot_wkr_task_grabing.incrementAndGet();
synchronized (taskReadyLock) {
while (seq_start == taskReadySeq) {
try {

View File

@ -201,6 +201,7 @@ public class TestSplitLogWorker {
try {
Thread.yield(); // let the worker start
Thread.sleep(1000);
waitForCounter(SplitLogCounters.tot_wkr_task_grabing, 0, 1, 1500);
// this time create a task node after starting the splitLogWorker
zkw.getRecoverableZooKeeper().create(PATH,
@ -231,6 +232,8 @@ public class TestSplitLogWorker {
try {
Thread.yield(); // let the worker start
Thread.sleep(100);
waitForCounter(SplitLogCounters.tot_wkr_task_grabing, 0, 1, 1500);
SplitLogTask unassignedManager = new SplitLogTask.Unassigned(MANAGER);
zkw.getRecoverableZooKeeper().create(PATH1, unassignedManager.toByteArray(),
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);