@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 testOrphanTaskAcquisition() throws Exception {
    LOG.info("TestOrphanTaskAcquisition");

    String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
    zkw.getRecoverableZooKeeper()
        .create(
            tasknode,
            TaskState.TASK_OWNED.get("dummy-worker"),
            Ids.OPEN_ACL_UNSAFE,
            CreateMode.PERSISTENT);

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

    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());
    waitForCounter(tot_mgr_heartbeat, 0, 1, 100);
    assertFalse(task.isUnassigned());
    long curt = System.currentTimeMillis();
    assertTrue((task.last_update <= curt) && (task.last_update > (curt - 1000)));
    LOG.info("waiting for manager to resubmit the orphan task");
    waitForCounter(tot_mgr_resubmit, 0, 1, to + 100);
    assertTrue(task.isUnassigned());
    waitForCounter(tot_mgr_rescan, 0, 1, to + 100);
  }
 @Test
 public void testEmptyLogDir() throws Exception {
   LOG.info("testEmptyLogDir");
   slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
   slm.finishInitialization();
   FileSystem fs = TEST_UTIL.getTestFileSystem();
   Path emptyLogDirPath = new Path(fs.getWorkingDirectory(), UUID.randomUUID().toString());
   fs.mkdirs(emptyLogDirPath);
   slm.splitLogDistributed(emptyLogDirPath);
   assertFalse(fs.exists(emptyLogDirPath));
 }
 @Test
 public void testVanishingTaskZNode() throws Exception {
   LOG.info("testVanishingTaskZNode");
   conf.setInt("hbase.splitlog.manager.unassigned.timeout", 0);
   slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
   slm.finishInitialization();
   FileSystem fs = TEST_UTIL.getTestFileSystem();
   final Path logDir = new Path(fs.getWorkingDirectory(), UUID.randomUUID().toString());
   fs.mkdirs(logDir);
   Path logFile = new Path(logDir, UUID.randomUUID().toString());
   fs.createNewFile(logFile);
   new Thread() {
     public void run() {
       try {
         // this call will block because there are no SplitLogWorkers
         slm.splitLogDistributed(logDir);
       } catch (Exception e) {
         LOG.warn("splitLogDistributed failed", e);
         fail();
       }
     }
   }.start();
   waitForCounter(tot_mgr_node_create_result, 0, 1, 10000);
   String znode = ZKSplitLog.getEncodedNodeName(zkw, logFile.toString());
   // remove the task znode
   ZKUtil.deleteNode(zkw, znode);
   waitForCounter(tot_mgr_get_data_nonode, 0, 1, 30000);
   waitForCounter(tot_mgr_log_split_batch_success, 0, 1, 1000);
   assertTrue(fs.exists(logFile));
   fs.delete(logDir, true);
 }
  @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());
  }
  private String submitTaskAndWait(TaskBatch batch, String name)
      throws KeeperException, InterruptedException {
    String tasknode = ZKSplitLog.getEncodedNodeName(zkw, name);
    NodeCreationListener listener = new NodeCreationListener(zkw, tasknode);
    zkw.registerListener(listener);
    ZKUtil.watchAndCheckExists(zkw, tasknode);

    slm.installTask(name, batch);
    assertEquals(1, batch.installed);
    assertTrue(slm.findOrCreateOrphanTask(tasknode).batch == batch);
    assertEquals(1L, tot_mgr_node_create_queued.get());

    LOG.debug("waiting for task node creation");
    listener.waitForCreation();
    LOG.debug("task created");
    return tasknode;
  }
  /**
   * Test whether the splitlog correctly creates a task in zookeeper
   *
   * @throws Exception
   */
  @Test
  public void testTaskCreation() throws Exception {
    LOG.info("TestTaskCreation - test the creation of a task in zk");

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

    String tasknode = submitTaskAndWait(batch, "foo/1");

    byte[] data = ZKUtil.getData(zkw, tasknode);
    LOG.info("Task node created " + new String(data));
    assertTrue(TaskState.TASK_UNASSIGNED.equals(data, "dummy-master"));
  }
  @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 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")));
  }
  @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);
  }
  @Test
  public void testUnassignedTimeout() throws Exception {
    LOG.info("TestUnassignedTimeout - iff all tasks are unassigned then" + " resubmit");

    // create an orphan task in OWNED state
    String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
    zkw.getRecoverableZooKeeper()
        .create(
            tasknode1,
            TaskState.TASK_OWNED.get("dummy-worker"),
            Ids.OPEN_ACL_UNSAFE,
            CreateMode.PERSISTENT);

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

    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
    slm.finishInitialization();
    waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);

    // submit another task which will stay in unassigned mode
    TaskBatch batch = new TaskBatch();
    submitTaskAndWait(batch, "foo/1");

    // keep updating the orphan owned node every to/2 seconds
    for (int i = 0; i < (3 * to) / 100; i++) {
      Thread.sleep(100);
      ZKUtil.setData(zkw, tasknode1, TaskState.TASK_OWNED.get("dummy-worker"));
    }

    // since we have stopped heartbeating the owned node therefore it should
    // get resubmitted
    LOG.info("waiting for manager to resubmit the orphan task");
    waitForCounter(tot_mgr_resubmit, 0, 1, to + 500);

    // now all the nodes are unassigned. manager should post another rescan
    waitForCounter(tot_mgr_resubmit_unassigned, 0, 1, 2 * to + 500);
  }
  @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;
  }
 @After
 public void teardown() throws IOException, KeeperException {
   stopper.stop("");
   slm.stop();
   TEST_UTIL.shutdownMiniZKCluster();
 }