/**
   * Submit a log split task to executor service
   *
   * @param curTask task to submit
   * @param curTaskZKVersion current version of task
   */
  void submitTask(
      final String curTask,
      final RecoveryMode mode,
      final int curTaskZKVersion,
      final int reportPeriod) {
    final MutableInt zkVersion = new MutableInt(curTaskZKVersion);

    CancelableProgressable reporter =
        new CancelableProgressable() {
          private long last_report_at = 0;

          @Override
          public boolean progress() {
            long t = EnvironmentEdgeManager.currentTime();
            if ((t - last_report_at) > reportPeriod) {
              last_report_at = t;
              int latestZKVersion =
                  attemptToOwnTask(
                      false, watcher, server.getServerName(), curTask, mode, zkVersion.intValue());
              if (latestZKVersion < 0) {
                LOG.warn("Failed to heartbeat the task" + curTask);
                return false;
              }
              zkVersion.setValue(latestZKVersion);
            }
            return true;
          }
        };
    ZkSplitLogWorkerCoordination.ZkSplitTaskDetails splitTaskDetails =
        new ZkSplitLogWorkerCoordination.ZkSplitTaskDetails();
    splitTaskDetails.setTaskNode(curTask);
    splitTaskDetails.setCurTaskZKVersion(zkVersion);

    WALSplitterHandler hsh =
        new WALSplitterHandler(
            server,
            this,
            splitTaskDetails,
            reporter,
            this.tasksInProgress,
            splitTaskExecutor,
            mode);
    server.getExecutorService().submit(hsh);
  }
  /**
   * try to grab a 'lock' on the task zk node to own and execute the task.
   *
   * <p>
   *
   * @param path zk node for the task
   */
  private void grabTask(String path) {
    Stat stat = new Stat();
    byte[] data;
    synchronized (grabTaskLock) {
      currentTask = path;
      workerInGrabTask = true;
      if (Thread.interrupted()) {
        return;
      }
    }
    try {
      try {
        if ((data = ZKUtil.getDataNoWatch(watcher, path, stat)) == null) {
          SplitLogCounters.tot_wkr_failed_to_grab_task_no_data.incrementAndGet();
          return;
        }
      } catch (KeeperException e) {
        LOG.warn("Failed to get data for znode " + path, e);
        SplitLogCounters.tot_wkr_failed_to_grab_task_exception.incrementAndGet();
        return;
      }
      SplitLogTask slt;
      try {
        slt = SplitLogTask.parseFrom(data);
      } catch (DeserializationException e) {
        LOG.warn("Failed parse data for znode " + path, e);
        SplitLogCounters.tot_wkr_failed_to_grab_task_exception.incrementAndGet();
        return;
      }
      if (!slt.isUnassigned()) {
        SplitLogCounters.tot_wkr_failed_to_grab_task_owned.incrementAndGet();
        return;
      }

      currentVersion =
          attemptToOwnTask(
              true, watcher, server.getServerName(), path, slt.getMode(), stat.getVersion());
      if (currentVersion < 0) {
        SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.incrementAndGet();
        return;
      }

      if (ZKSplitLog.isRescanNode(watcher, currentTask)) {
        ZkSplitLogWorkerCoordination.ZkSplitTaskDetails splitTaskDetails =
            new ZkSplitLogWorkerCoordination.ZkSplitTaskDetails();
        splitTaskDetails.setTaskNode(currentTask);
        splitTaskDetails.setCurTaskZKVersion(new MutableInt(currentVersion));

        endTask(
            new SplitLogTask.Done(server.getServerName(), slt.getMode()),
            SplitLogCounters.tot_wkr_task_acquired_rescan,
            splitTaskDetails);
        return;
      }

      LOG.info("worker " + server.getServerName() + " acquired task " + path);
      SplitLogCounters.tot_wkr_task_acquired.incrementAndGet();
      getDataSetWatchAsync();

      submitTask(path, slt.getMode(), currentVersion, reportPeriod);

      // after a successful submit, sleep a little bit to allow other RSs to grab the rest tasks
      try {
        int sleepTime = RandomUtils.nextInt(500) + 500;
        Thread.sleep(sleepTime);
      } catch (InterruptedException e) {
        LOG.warn("Interrupted while yielding for other region servers", e);
        Thread.currentThread().interrupt();
      }
    } finally {
      synchronized (grabTaskLock) {
        workerInGrabTask = false;
        // clear the interrupt from stopTask() otherwise the next task will
        // suffer
        Thread.interrupted();
      }
    }
  }