HBASE-8954 TestSplitLogWorker#testPreemptTask failed
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1505888 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
cf3da420d0
commit
bc34f88ca2
@ -33,6 +33,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||||
import org.apache.hadoop.hbase.SplitLogTask;
|
import org.apache.hadoop.hbase.SplitLogTask;
|
||||||
@ -169,8 +170,14 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||||||
try {
|
try {
|
||||||
LOG.info("SplitLogWorker " + this.serverName + " starting");
|
LOG.info("SplitLogWorker " + this.serverName + " starting");
|
||||||
this.watcher.registerListener(this);
|
this.watcher.registerListener(this);
|
||||||
// initialize a new connection for splitlogworker configuration
|
boolean distributedLogReplay = this.conf.getBoolean(
|
||||||
HConnectionManager.getConnection(conf);
|
HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
|
||||||
|
HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
|
||||||
|
if (distributedLogReplay) {
|
||||||
|
// initialize a new connection for splitlogworker configuration
|
||||||
|
HConnectionManager.getConnection(conf);
|
||||||
|
}
|
||||||
|
|
||||||
// wait for master to create the splitLogZnode
|
// wait for master to create the splitLogZnode
|
||||||
int res = -1;
|
int res = -1;
|
||||||
while (res == -1 && !exitWorker) {
|
while (res == -1 && !exitWorker) {
|
||||||
|
@ -100,7 +100,6 @@ public class TestSplitLogWorker {
|
|||||||
assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
|
assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
|
||||||
LOG.debug(zkw.splitLogZNode + " created");
|
LOG.debug(zkw.splitLogZNode + " created");
|
||||||
SplitLogCounters.resetCounters();
|
SplitLogCounters.resetCounters();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
@ -201,7 +200,7 @@ public class TestSplitLogWorker {
|
|||||||
try {
|
try {
|
||||||
Thread.yield(); // let the worker start
|
Thread.yield(); // let the worker start
|
||||||
Thread.sleep(1000);
|
Thread.sleep(1000);
|
||||||
waitForCounter(SplitLogCounters.tot_wkr_task_grabing, 0, 1, 1500);
|
waitForCounter(SplitLogCounters.tot_wkr_task_grabing, 0, 1, 5000);
|
||||||
|
|
||||||
// this time create a task node after starting the splitLogWorker
|
// this time create a task node after starting the splitLogWorker
|
||||||
zkw.getRecoverableZooKeeper().create(PATH,
|
zkw.getRecoverableZooKeeper().create(PATH,
|
||||||
|
Loading…
x
Reference in New Issue
Block a user