HBASE-3819 TestSplitLogWorker has too many SLWs running -- makes for contention and occasional failures; attempt at fixing broken tests in build introduced by first commit on this issue

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1096859 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-04-26 19:23:00 +00:00
parent 992bf057e7
commit 95a45d1341
2 changed files with 28 additions and 21 deletions

View File

@ -136,7 +136,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
@Override
public void run() {
LOG.info("SplitLogWorker starting");
LOG.info("SplitLogWorker " + this.serverName + " starting");
this.watcher.registerListener(this);
int res;
// wait for master to create the splitLogZnode
@ -163,7 +163,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
taskLoop();
LOG.info("SplitLogWorker exiting");
LOG.info("SplitLogWorker " + this.serverName + " exiting");
}
/**

View File

@ -19,7 +19,12 @@
*/
package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.resetCounters;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_failed_to_grab_task_lost_race;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_failed_to_grab_task_owned;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_preempt_task;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired_rescan;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@ -39,9 +44,7 @@ import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs.Ids;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -58,6 +61,12 @@ public class TestSplitLogWorker {
private void waitForCounter(AtomicLong ctr, long oldval, long newval,
long timems) {
assertTrue("ctr=" + ctr.get() + ", oldval=" + oldval + ", newval=" + newval,
waitForCounterBoolean(ctr, oldval, newval, timems));
}
private boolean waitForCounterBoolean(AtomicLong ctr, long oldval, long newval,
long timems) {
assert ctr.get() == oldval;
long curt = System.currentTimeMillis();
long endt = curt + timems;
@ -70,19 +79,10 @@ public class TestSplitLogWorker {
curt = System.currentTimeMillis();
} else {
assertEquals(newval, ctr.get());
return;
return true;
}
}
assertTrue("ctr=" + ctr.get() + ", oldval=" + oldval + ", newval=" + newval,
false);
}
@BeforeClass
public static void setUpBeforeClass() throws Exception {
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
return false;
}
@Before
@ -170,14 +170,21 @@ public class TestSplitLogWorker {
"svr2", neverEndingTask);
slw1.start();
slw2.start();
waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
waitForCounter(tot_wkr_failed_to_grab_task_lost_race, 0, 1, 1000);
assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
try {
waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
boolean first =
waitForCounterBoolean(tot_wkr_failed_to_grab_task_owned, 0, 1, 1000);
boolean second =
waitForCounterBoolean(tot_wkr_failed_to_grab_task_lost_race, 0, 1, 100);
assertTrue(first || second);
assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
ZKSplitLog.getEncodedNodeName(zkw, "trft")), "svr1") ||
TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
ZKSplitLog.getEncodedNodeName(zkw, "trft")), "svr2"));
stopSplitLogWorker(slw1);
stopSplitLogWorker(slw2);
} finally {
stopSplitLogWorker(slw1);
stopSplitLogWorker(slw2);
}
}
@Test