/**
  * Test that if we fail a flush, abort gets set on close.
  *
  * @see <a href="https://issues.apache.org/jira/browse/HBASE-4270">HBASE-4270</a>
  * @throws IOException
  * @throws NodeExistsException
  * @throws KeeperException
  */
 @Test
 public void testFailedFlushAborts() throws IOException, NodeExistsException, KeeperException {
   final Server server = new MockServer(HTU, false);
   final RegionServerServices rss = HTU.createMockRegionServerService();
   HTableDescriptor htd = TEST_HTD;
   final HRegionInfo hri =
       new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW, HConstants.EMPTY_END_ROW);
   HRegion region = HTU.createLocalHRegion(hri, htd);
   try {
     assertNotNull(region);
     // Spy on the region so can throw exception when close is called.
     HRegion spy = Mockito.spy(region);
     final boolean abort = false;
     Mockito.when(spy.close(abort)).thenThrow(new RuntimeException("Mocked failed close!"));
     // The CloseRegionHandler will try to get an HRegion that corresponds
     // to the passed hri -- so insert the region into the online region Set.
     rss.addToOnlineRegions(spy);
     // Assert the Server is NOT stopped before we call close region.
     assertFalse(server.isStopped());
     CloseRegionHandler handler = new CloseRegionHandler(server, rss, hri, false, false, -1);
     boolean throwable = false;
     try {
       handler.process();
     } catch (Throwable t) {
       throwable = true;
     } finally {
       assertTrue(throwable);
       // Abort calls stop so stopped flag should be set.
       assertTrue(server.isStopped());
     }
   } finally {
     HRegion.closeHRegion(region);
   }
 }
 private void OpenRegion(
     Server server, RegionServerServices rss, HTableDescriptor htd, HRegionInfo hri)
     throws IOException, NodeExistsException, KeeperException, DeserializationException {
   // Create it OFFLINE node, which is what Master set before sending OPEN RPC
   ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
   OpenRegionHandler openHandler = new OpenRegionHandler(server, rss, hri, htd);
   rss.getRegionsInTransitionInRS().put(hri.getEncodedNameAsBytes(), Boolean.TRUE);
   openHandler.process();
   // This parse is not used?
   RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
   // delete the node, which is what Master do after the region is opened
   ZKAssign.deleteNode(
       server.getZooKeeper(),
       hri.getEncodedName(),
       EventType.RS_ZK_REGION_OPENED,
       server.getServerName());
 }
  /**
   * 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);
  }
 /**
  * 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;
               }
             }
           }
         }
       }
     }
   }
 }
  /**
   * 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();
      }
    }
  }