@Before
  public void setup() throws Exception {
    TEST_UTIL = new HBaseTestingUtility();
    TEST_UTIL.startMiniZKCluster();
    conf = TEST_UTIL.getConfiguration();
    // Use a different ZK wrapper instance for each tests.
    zkw =
        new ZooKeeperWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null);
    ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode);
    ZKUtil.createAndFailSilent(zkw, zkw.baseZNode);
    assertTrue(ZKUtil.checkExists(zkw, zkw.baseZNode) != -1);
    LOG.debug(zkw.baseZNode + " created");
    ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
    assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
    LOG.debug(zkw.splitLogZNode + " created");

    stopped = false;
    resetCounters();

    // By default, we let the test manage the error as before, so the server
    //  does not appear as dead from the master point of view, only from the split log pov.
    Mockito.when(sm.isServerOnline(Mockito.any(ServerName.class))).thenReturn(true);
    Mockito.when(master.getServerManager()).thenReturn(sm);

    to = 4000;
    conf.setInt("hbase.splitlog.manager.timeout", to);
    conf.setInt("hbase.splitlog.manager.unassigned.timeout", 2 * to);
    conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
    to = to + 4 * 100;
  }
  @Test
  public void testUnassignedOrphan() throws Exception {
    LOG.info("TestUnassignedOrphan - an unassigned task is resubmitted at" + " startup");
    String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
    // create an unassigned orphan task
    zkw.getRecoverableZooKeeper()
        .create(
            tasknode,
            TaskState.TASK_UNASSIGNED.get("dummy-worker"),
            Ids.OPEN_ACL_UNSAFE,
            CreateMode.PERSISTENT);
    int version = ZKUtil.checkExists(zkw, tasknode);

    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
    slm.finishInitialization();
    waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
    Task task = slm.findOrCreateOrphanTask(tasknode);
    assertTrue(task.isOrphan());
    assertTrue(task.isUnassigned());
    // wait for RESCAN node to be created
    waitForCounter(tot_mgr_rescan, 0, 1, 500);
    Task task2 = slm.findOrCreateOrphanTask(tasknode);
    assertTrue(task == task2);
    LOG.debug("task = " + task);
    assertEquals(1L, tot_mgr_resubmit.get());
    assertEquals(1, task.incarnation);
    assertEquals(0, task.unforcedResubmits);
    assertTrue(task.isOrphan());
    assertTrue(task.isUnassigned());
    assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);
  }
  @Test
  public void testTaskResigned() throws Exception {
    LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");
    assertEquals(tot_mgr_resubmit.get(), 0);
    slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER, null);
    slm.finishInitialization();
    assertEquals(tot_mgr_resubmit.get(), 0);
    TaskBatch batch = new TaskBatch();
    String tasknode = submitTaskAndWait(batch, "foo/1");
    assertEquals(tot_mgr_resubmit.get(), 0);
    final ServerName worker1 = new ServerName("worker1,1,1");
    assertEquals(tot_mgr_resubmit.get(), 0);
    SplitLogTask slt = new SplitLogTask.Resigned(worker1);
    assertEquals(tot_mgr_resubmit.get(), 0);
    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
    int version = ZKUtil.checkExists(zkw, tasknode);
    // Could be small race here.
    if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, to / 2);
    assertEquals(tot_mgr_resubmit.get(), 1);
    int version1 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue("version1=" + version1 + ", version=" + version, version1 > version);

    byte[] taskstate = ZKUtil.getData(zkw, tasknode);
    slt = SplitLogTask.parseFrom(taskstate);
    assertTrue(slt.isUnassigned(DUMMY_MASTER));
  }
  @Test
  public void testDeadWorker() throws Exception {
    LOG.info("testDeadWorker");

    conf.setLong("hbase.splitlog.max.resubmit", 0);
    slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER, null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();

    String tasknode = submitTaskAndWait(batch, "foo/1");
    int version = ZKUtil.checkExists(zkw, tasknode);
    final ServerName worker1 = new ServerName("worker1,1,1");
    SplitLogTask slt = new SplitLogTask.Owned(worker1);
    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
    if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to / 2);
    slm.handleDeadWorker(worker1);
    if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, to + to / 2);
    if (tot_mgr_resubmit_dead_server_task.get() == 0) {
      waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, to + to / 2);
    }

    int version1 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue(version1 > version);
    byte[] taskstate = ZKUtil.getData(zkw, tasknode);
    slt = SplitLogTask.parseFrom(taskstate);
    assertTrue(slt.isUnassigned(DUMMY_MASTER));
    return;
  }
  @Test
  public void testRescanCleanup() throws Exception {
    LOG.info("TestRescanCleanup - ensure RESCAN nodes are cleaned up");

    slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER, null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();

    String tasknode = submitTaskAndWait(batch, "foo/1");
    int version = ZKUtil.checkExists(zkw, tasknode);
    final ServerName worker1 = new ServerName("worker1,1,1");
    SplitLogTask slt = new SplitLogTask.Owned(worker1);
    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
    waitForCounter(tot_mgr_heartbeat, 0, 1, to / 2);
    waitForCounter(
        new Expr() {
          @Override
          public long eval() {
            return (tot_mgr_resubmit.get() + tot_mgr_resubmit_failed.get());
          }
        },
        0,
        1,
        5 * 60000); // wait long enough
    Assert.assertEquals(
        "Could not run test. Lost ZK connection?", 0, tot_mgr_resubmit_failed.get());
    int version1 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue(version1 > version);
    byte[] taskstate = ZKUtil.getData(zkw, tasknode);
    slt = SplitLogTask.parseFrom(taskstate);
    assertTrue(slt.isUnassigned(DUMMY_MASTER));

    waitForCounter(tot_mgr_rescan_deleted, 0, 1, to / 2);
  }
  @Test
  public void testMultipleResubmits() throws Exception {
    LOG.info("TestMultipleResbmits - no indefinite resubmissions");

    conf.setInt("hbase.splitlog.max.resubmit", 2);
    slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER, null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();

    String tasknode = submitTaskAndWait(batch, "foo/1");
    int version = ZKUtil.checkExists(zkw, tasknode);
    final ServerName worker1 = new ServerName("worker1,1,1");
    final ServerName worker2 = new ServerName("worker2,1,1");
    final ServerName worker3 = new ServerName("worker3,1,1");
    SplitLogTask slt = new SplitLogTask.Owned(worker1);
    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
    waitForCounter(tot_mgr_heartbeat, 0, 1, to / 2);
    waitForCounter(tot_mgr_resubmit, 0, 1, to + to / 2);
    int version1 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue(version1 > version);
    slt = new SplitLogTask.Owned(worker2);
    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
    waitForCounter(tot_mgr_heartbeat, 1, 2, to / 2);
    waitForCounter(tot_mgr_resubmit, 1, 2, to + to / 2);
    int version2 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue(version2 > version1);
    slt = new SplitLogTask.Owned(worker3);
    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
    waitForCounter(tot_mgr_heartbeat, 1, 2, to / 2);
    waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + to / 2);
    Thread.sleep(to + to / 2);
    assertEquals(2L, tot_mgr_resubmit.get());
  }
  @Test
  public void testMultipleResubmits() throws Exception {
    LOG.info("TestMultipleResbmits - no indefinite resubmissions");

    int to = 1000;
    conf.setInt("hbase.splitlog.manager.timeout", to);
    conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
    to = to + 2 * 100;

    conf.setInt("hbase.splitlog.max.resubmit", 2);
    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();

    String tasknode = submitTaskAndWait(batch, "foo/1");
    int version = ZKUtil.checkExists(zkw, tasknode);

    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
    waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
    waitForCounter(tot_mgr_resubmit, 0, 1, to + 100);
    int version1 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue(version1 > version);
    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker2"));
    waitForCounter(tot_mgr_heartbeat, 1, 2, 1000);
    waitForCounter(tot_mgr_resubmit, 1, 2, to + 100);
    int version2 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue(version2 > version1);
    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker3"));
    waitForCounter(tot_mgr_heartbeat, 1, 2, 1000);
    waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + 100);
    Thread.sleep(to + 100);
    assertEquals(2L, tot_mgr_resubmit.get());
  }
  @Before
  public void setup() throws Exception {
    TEST_UTIL.startMiniZKCluster();
    conf = TEST_UTIL.getConfiguration();
    zkw = new ZooKeeperWatcher(conf, "split-log-manager-tests", null);
    ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode);
    ZKUtil.createAndFailSilent(zkw, zkw.baseZNode);
    assertTrue(ZKUtil.checkExists(zkw, zkw.baseZNode) != -1);
    LOG.debug(zkw.baseZNode + " created");
    ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
    assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
    LOG.debug(zkw.splitLogZNode + " created");

    stopped = false;
    resetCounters();
  }
  @Test
  public void testTaskResigned() throws Exception {
    LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");

    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();
    String tasknode = submitTaskAndWait(batch, "foo/1");
    ZKUtil.setData(zkw, tasknode, TaskState.TASK_RESIGNED.get("worker"));
    int version = ZKUtil.checkExists(zkw, tasknode);

    waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
    int version1 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue(version1 > version);

    byte[] taskstate = ZKUtil.getData(zkw, tasknode);
    assertTrue(Arrays.equals(taskstate, TaskState.TASK_UNASSIGNED.get("dummy-master")));
  }
 @Override
 public void init() throws ReplicationException {
   try {
     if (ZKUtil.checkExists(this.zookeeper, this.peersZNode) < 0) {
       ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
     }
   } catch (KeeperException e) {
     throw new ReplicationException("Could not initialize replication peers", e);
   }
   connectExistingPeers();
 }
Beispiel #11
0
  @Test
  public void testRemoveStaleRecoveringRegionsDuringMasterInitialization() throws Exception {
    // this test is for when distributed log replay is enabled
    if (!UTIL.getConfiguration().getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false)) return;

    LOG.info("Starting testRemoveStaleRecoveringRegionsDuringMasterInitialization");
    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
    MasterFileSystem fs = master.getMasterFileSystem();

    String failedRegion = "failedRegoin1";
    String staleRegion = "staleRegion";
    ServerName inRecoveryServerName = ServerName.valueOf("mgr,1,1");
    ServerName previouselyFaildServerName = ServerName.valueOf("previous,1,1");
    String walPath =
        "/hbase/data/.logs/" + inRecoveryServerName.getServerName() + "-splitting/test";
    // Create a ZKW to use in the test
    ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(UTIL);
    zkw.getRecoverableZooKeeper()
        .create(
            ZKSplitLog.getEncodedNodeName(zkw, walPath),
            new SplitLogTask.Owned(inRecoveryServerName, fs.getLogRecoveryMode()).toByteArray(),
            Ids.OPEN_ACL_UNSAFE,
            CreateMode.PERSISTENT);
    String staleRegionPath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, staleRegion);
    ZKUtil.createWithParents(zkw, staleRegionPath);
    String inRecoveringRegionPath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, failedRegion);
    inRecoveringRegionPath =
        ZKUtil.joinZNode(inRecoveringRegionPath, inRecoveryServerName.getServerName());
    ZKUtil.createWithParents(zkw, inRecoveringRegionPath);
    Set<ServerName> servers = new HashSet<ServerName>();
    servers.add(previouselyFaildServerName);
    fs.removeStaleRecoveringRegionsFromZK(servers);

    // verification
    assertFalse(ZKUtil.checkExists(zkw, staleRegionPath) != -1);
    assertTrue(ZKUtil.checkExists(zkw, inRecoveringRegionPath) != -1);

    ZKUtil.deleteChildrenRecursively(zkw, zkw.recoveringRegionsZNode);
    ZKUtil.deleteChildrenRecursively(zkw, zkw.splitLogZNode);
    zkw.close();
  }
  @Test
  public void testRescanCleanup() throws Exception {
    LOG.info("TestRescanCleanup - ensure RESCAN nodes are cleaned up");

    conf.setInt("hbase.splitlog.manager.timeout", 1000);
    conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();

    String tasknode = submitTaskAndWait(batch, "foo/1");
    int version = ZKUtil.checkExists(zkw, tasknode);

    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
    waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
    waitForCounter(
        new Expr() {
          @Override
          public long eval() {
            return (tot_mgr_resubmit.get() + tot_mgr_resubmit_failed.get());
          }
        },
        0,
        1,
        5 * 60000); // wait long enough
    if (tot_mgr_resubmit_failed.get() == 0) {
      int version1 = ZKUtil.checkExists(zkw, tasknode);
      assertTrue(version1 > version);
      byte[] taskstate = ZKUtil.getData(zkw, tasknode);
      assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"), taskstate));

      waitForCounter(tot_mgr_rescan_deleted, 0, 1, 1000);
    } else {
      LOG.warn("Could not run test. Lost ZK connection?");
    }

    return;
  }
 @Override
 public boolean isReady() throws InterruptedException {
   int result = -1;
   try {
     result = ZKUtil.checkExists(watcher, watcher.splitLogZNode);
   } catch (KeeperException e) {
     // ignore
     LOG.warn("Exception when checking for " + watcher.splitLogZNode + " ... retrying", e);
   }
   if (result == -1) {
     LOG.info(watcher.splitLogZNode + " znode does not exist, waiting for master to create");
     Thread.sleep(1000);
   }
   return (result != -1);
 }
  @Test
  public void testDeadWorker() throws Exception {
    LOG.info("testDeadWorker");

    conf.setLong("hbase.splitlog.max.resubmit", 0);
    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();

    String tasknode = submitTaskAndWait(batch, "foo/1");
    int version = ZKUtil.checkExists(zkw, tasknode);

    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
    waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
    slm.handleDeadWorker("worker1");
    waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
    waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, 1000);

    int version1 = ZKUtil.checkExists(zkw, tasknode);
    assertTrue(version1 > version);
    byte[] taskstate = ZKUtil.getData(zkw, tasknode);
    assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"), taskstate));
    return;
  }
  @Test
  public void testTaskDone() throws Exception {
    LOG.info("TestTaskDone - cleanup task node once in DONE state");

    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();
    String tasknode = submitTaskAndWait(batch, "foo/1");
    ZKUtil.setData(zkw, tasknode, TaskState.TASK_DONE.get("worker"));
    synchronized (batch) {
      while (batch.installed != batch.done) {
        batch.wait();
      }
    }
    waitForCounter(tot_mgr_task_deleted, 0, 1, 1000);
    assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
  }
  @Test
  public void testTaskDone() throws Exception {
    LOG.info("TestTaskDone - cleanup task node once in DONE state");

    slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER, null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();
    String tasknode = submitTaskAndWait(batch, "foo/1");
    final ServerName worker1 = new ServerName("worker1,1,1");
    SplitLogTask slt = new SplitLogTask.Done(worker1);
    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
    synchronized (batch) {
      while (batch.installed != batch.done) {
        batch.wait();
      }
    }
    waitForCounter(tot_mgr_task_deleted, 0, 1, to / 2);
    assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
  }
 @Override
 public void setPeerTableCFsConfig(String id, String tableCFsStr) throws ReplicationException {
   try {
     if (!peerExists(id)) {
       throw new IllegalArgumentException(
           "Cannot set peer tableCFs because id=" + id + " does not exist.");
     }
     String tableCFsZKNode = getTableCFsNode(id);
     byte[] tableCFs = Bytes.toBytes(tableCFsStr);
     if (ZKUtil.checkExists(this.zookeeper, tableCFsZKNode) != -1) {
       ZKUtil.setData(this.zookeeper, tableCFsZKNode, tableCFs);
     } else {
       ZKUtil.createAndWatch(this.zookeeper, tableCFsZKNode, tableCFs);
     }
     LOG.info("Peer tableCFs with id= " + id + " is now " + tableCFsStr);
   } catch (KeeperException e) {
     throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e);
   }
 }
  @Test
  public void testTaskErr() throws Exception {
    LOG.info("TestTaskErr - cleanup task node once in ERR state");

    conf.setInt("hbase.splitlog.max.resubmit", 0);
    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();

    String tasknode = submitTaskAndWait(batch, "foo/1");
    ZKUtil.setData(zkw, tasknode, TaskState.TASK_ERR.get("worker"));
    synchronized (batch) {
      while (batch.installed != batch.error) {
        batch.wait();
      }
    }
    waitForCounter(tot_mgr_task_deleted, 0, 1, 1000);
    assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
    conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLog.DEFAULT_MAX_RESUBMIT);
  }
 /**
  * Update the state znode of a peer cluster.
  *
  * @param id
  * @param state
  */
 private void changePeerState(String id, ZooKeeperProtos.ReplicationState.State state)
     throws ReplicationException {
   try {
     if (!peerExists(id)) {
       throw new IllegalArgumentException(
           "Cannot enable/disable peer because id=" + id + " does not exist.");
     }
     String peerStateZNode = getPeerStateNode(id);
     byte[] stateBytes =
         (state == ZooKeeperProtos.ReplicationState.State.ENABLED)
             ? ENABLED_ZNODE_BYTES
             : DISABLED_ZNODE_BYTES;
     if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
       ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes);
     } else {
       ZKUtil.createAndWatch(this.zookeeper, peerStateZNode, stateBytes);
     }
     LOG.info("Peer with id= " + id + " is now " + state.name());
   } catch (KeeperException e) {
     throw new ReplicationException("Unable to change state of the peer with id=" + id, e);
   }
 }
  public boolean shouldAddCheckerMaster() {

    ZooKeeperWatcher zk = super.getZooKeeper();
    String numberN = ZKUtil.joinZNode(zk.baseZNode, CCIndexConstants.CheckNumNode);
    try {
      if (ZKUtil.checkExists(zk, numberN) != -1) {
        ZKUtil.createSetData(zk, numberN, Bytes.toBytes(1));
      } else {
        int num = Bytes.toInt(ZKUtil.getData(zk, numberN));
        if (num < this.checkMasterN) {
          ZKUtil.setData(zk, numberN, Bytes.toBytes(num + 1));
          return true;
        } else {
          return false;
        }
      }
    } catch (KeeperException e) {
      // TODO Auto-generated catch block
      e.printStackTrace();
    }

    return false;
  }
  @Test
  public void testTaskErr() throws Exception {
    LOG.info("TestTaskErr - cleanup task node once in ERR state");

    conf.setInt("hbase.splitlog.max.resubmit", 0);
    slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER, null);
    slm.finishInitialization();
    TaskBatch batch = new TaskBatch();

    String tasknode = submitTaskAndWait(batch, "foo/1");
    final ServerName worker1 = new ServerName("worker1,1,1");
    SplitLogTask slt = new SplitLogTask.Err(worker1);
    ZKUtil.setData(zkw, tasknode, slt.toByteArray());

    synchronized (batch) {
      while (batch.installed != batch.error) {
        batch.wait();
      }
    }
    waitForCounter(tot_mgr_task_deleted, 0, 1, to / 2);
    assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
    conf.setInt("hbase.splitlog.max.resubmit", SplitLogManager.DEFAULT_MAX_RESUBMIT);
  }
 /**
  * Wait for tasks to become available at /hbase/splitlog zknode. Grab a task one at a time. This
  * policy puts an upper-limit on the number of simultaneous log splitting that could be happening
  * in a cluster.
  *
  * <p>Synchronization using {@link #taskReadyLock} ensures that it will try to grab every task
  * that has been put up
  *
  * @throws InterruptedException
  */
 @Override
 public void taskLoop() throws InterruptedException {
   while (!shouldStop) {
     int seq_start = taskReadySeq;
     List<String> paths = null;
     paths = getTaskList();
     if (paths == null) {
       LOG.warn(
           "Could not get tasks, did someone remove "
               + watcher.splitLogZNode
               + " ... worker thread exiting.");
       return;
     }
     // pick meta wal firstly
     int offset = (int) (Math.random() * paths.size());
     for (int i = 0; i < paths.size(); i++) {
       if (DefaultWALProvider.isMetaFile(paths.get(i))) {
         offset = i;
         break;
       }
     }
     int numTasks = paths.size();
     for (int i = 0; i < numTasks; i++) {
       int idx = (i + offset) % paths.size();
       // don't call ZKSplitLog.getNodeName() because that will lead to
       // double encoding of the path name
       if (this.calculateAvailableSplitters(numTasks) > 0) {
         grabTask(ZKUtil.joinZNode(watcher.splitLogZNode, paths.get(idx)));
       } else {
         LOG.debug(
             "Current region server "
                 + server.getServerName()
                 + " has "
                 + this.tasksInProgress.get()
                 + " tasks in progress and can't take more.");
         break;
       }
       if (shouldStop) {
         return;
       }
     }
     SplitLogCounters.tot_wkr_task_grabing.incrementAndGet();
     synchronized (taskReadyLock) {
       while (seq_start == taskReadySeq) {
         taskReadyLock.wait(checkInterval);
         if (server != null) {
           // check to see if we have stale recovering regions in our internal memory state
           Map<String, HRegion> recoveringRegions = server.getRecoveringRegions();
           if (!recoveringRegions.isEmpty()) {
             // Make a local copy to prevent ConcurrentModificationException when other threads
             // modify recoveringRegions
             List<String> tmpCopy = new ArrayList<String>(recoveringRegions.keySet());
             int listSize = tmpCopy.size();
             for (int i = 0; i < listSize; i++) {
               String region = tmpCopy.get(i);
               String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region);
               try {
                 if (ZKUtil.checkExists(watcher, nodePath) == -1) {
                   HRegion r = recoveringRegions.remove(region);
                   if (r != null) {
                     r.setRecovering(false);
                   }
                   LOG.debug("Mark recovering region:" + region + " up.");
                 } else {
                   // current check is a defensive(or redundant) mechanism to prevent us from
                   // having stale recovering regions in our internal RS memory state while
                   // zookeeper(source of truth) says differently. We stop at the first good one
                   // because we should not have a single instance such as this in normal case so
                   // check the first one is good enough.
                   break;
                 }
               } catch (KeeperException e) {
                 // ignore zookeeper error
                 LOG.debug("Got a zookeeper when trying to open a recovering region", e);
                 break;
               }
             }
           }
         }
       }
     }
   }
 }