001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.procedure2;
019
020import edu.umd.cs.findbugs.annotations.Nullable;
021import java.io.IOException;
022import java.io.UncheckedIOException;
023import java.util.ArrayDeque;
024import java.util.ArrayList;
025import java.util.Arrays;
026import java.util.Collection;
027import java.util.Deque;
028import java.util.HashSet;
029import java.util.List;
030import java.util.Set;
031import java.util.concurrent.ConcurrentHashMap;
032import java.util.concurrent.CopyOnWriteArrayList;
033import java.util.concurrent.Executor;
034import java.util.concurrent.Executors;
035import java.util.concurrent.TimeUnit;
036import java.util.concurrent.atomic.AtomicBoolean;
037import java.util.concurrent.atomic.AtomicInteger;
038import java.util.concurrent.atomic.AtomicLong;
039import java.util.stream.Collectors;
040import java.util.stream.Stream;
041import org.apache.hadoop.conf.Configuration;
042import org.apache.hadoop.hbase.HConstants;
043import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
044import org.apache.hadoop.hbase.log.HBaseMarkers;
045import org.apache.hadoop.hbase.procedure2.Procedure.LockState;
046import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
047import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
048import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureStoreListener;
049import org.apache.hadoop.hbase.procedure2.util.StringUtils;
050import org.apache.hadoop.hbase.security.User;
051import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
052import org.apache.hadoop.hbase.util.IdLock;
053import org.apache.hadoop.hbase.util.NonceKey;
054import org.apache.hadoop.hbase.util.Threads;
055import org.apache.yetus.audience.InterfaceAudience;
056import org.slf4j.Logger;
057import org.slf4j.LoggerFactory;
058
059import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
060import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
061
062import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
063
064/**
065 * Thread Pool that executes the submitted procedures.
066 * The executor has a ProcedureStore associated.
067 * Each operation is logged and on restart the pending procedures are resumed.
068 *
069 * Unless the Procedure code throws an error (e.g. invalid user input)
070 * the procedure will complete (at some point in time), On restart the pending
071 * procedures are resumed and the once failed will be rolledback.
072 *
073 * The user can add procedures to the executor via submitProcedure(proc)
074 * check for the finished state via isFinished(procId)
075 * and get the result via getResult(procId)
076 */
077@InterfaceAudience.Private
078public class ProcedureExecutor<TEnvironment> {
079  private static final Logger LOG = LoggerFactory.getLogger(ProcedureExecutor.class);
080
081  public static final String CHECK_OWNER_SET_CONF_KEY = "hbase.procedure.check.owner.set";
082  private static final boolean DEFAULT_CHECK_OWNER_SET = false;
083
084  public static final String WORKER_KEEP_ALIVE_TIME_CONF_KEY =
085      "hbase.procedure.worker.keep.alive.time.msec";
086  private static final long DEFAULT_WORKER_KEEP_ALIVE_TIME = TimeUnit.MINUTES.toMillis(1);
087
088  public static final String EVICT_TTL_CONF_KEY = "hbase.procedure.cleaner.evict.ttl";
089  static final int DEFAULT_EVICT_TTL = 15 * 60000; // 15min
090
091  public static final String EVICT_ACKED_TTL_CONF_KEY ="hbase.procedure.cleaner.acked.evict.ttl";
092  static final int DEFAULT_ACKED_EVICT_TTL = 5 * 60000; // 5min
093
094  /**
095   * {@link #testing} is non-null when ProcedureExecutor is being tested. Tests will try to
096   * break PE having it fail at various junctures. When non-null, testing is set to an instance of
097   * the below internal {@link Testing} class with flags set for the particular test.
098   */
099  volatile Testing testing = null;
100
101  /**
102   * Class with parameters describing how to fail/die when in testing-context.
103   */
104  public static class Testing {
105    protected volatile boolean killIfHasParent = true;
106    protected volatile boolean killIfSuspended = false;
107
108    /**
109     * Kill the PE BEFORE we store state to the WAL. Good for figuring out if a Procedure is
110     * persisting all the state it needs to recover after a crash.
111     */
112    protected volatile boolean killBeforeStoreUpdate = false;
113    protected volatile boolean toggleKillBeforeStoreUpdate = false;
114
115    /**
116     * Set when we want to fail AFTER state has been stored into the WAL. Rarely used. HBASE-20978
117     * is about a case where memory-state was being set after store to WAL where a crash could
118     * cause us to get stuck. This flag allows killing at what was a vulnerable time.
119     */
120    protected volatile boolean killAfterStoreUpdate = false;
121    protected volatile boolean toggleKillAfterStoreUpdate = false;
122
123    protected boolean shouldKillBeforeStoreUpdate() {
124      final boolean kill = this.killBeforeStoreUpdate;
125      if (this.toggleKillBeforeStoreUpdate) {
126        this.killBeforeStoreUpdate = !kill;
127        LOG.warn("Toggle KILL before store update to: " + this.killBeforeStoreUpdate);
128      }
129      return kill;
130    }
131
132    protected boolean shouldKillBeforeStoreUpdate(boolean isSuspended, boolean hasParent) {
133      if (isSuspended && !killIfSuspended) {
134        return false;
135      }
136      if (hasParent && !killIfHasParent) {
137        return false;
138      }
139      return shouldKillBeforeStoreUpdate();
140    }
141
142    protected boolean shouldKillAfterStoreUpdate() {
143      final boolean kill = this.killAfterStoreUpdate;
144      if (this.toggleKillAfterStoreUpdate) {
145        this.killAfterStoreUpdate = !kill;
146        LOG.warn("Toggle KILL after store update to: " + this.killAfterStoreUpdate);
147      }
148      return kill;
149    }
150
151    protected boolean shouldKillAfterStoreUpdate(final boolean isSuspended) {
152      return (isSuspended && !killIfSuspended) ? false : shouldKillAfterStoreUpdate();
153    }
154  }
155
156  public interface ProcedureExecutorListener {
157    void procedureLoaded(long procId);
158    void procedureAdded(long procId);
159    void procedureFinished(long procId);
160  }
161
162  /**
163   * Map the the procId returned by submitProcedure(), the Root-ProcID, to the Procedure.
164   * Once a Root-Procedure completes (success or failure), the result will be added to this map.
165   * The user of ProcedureExecutor should call getResult(procId) to get the result.
166   */
167  private final ConcurrentHashMap<Long, CompletedProcedureRetainer<TEnvironment>> completed =
168    new ConcurrentHashMap<>();
169
170  /**
171   * Map the the procId returned by submitProcedure(), the Root-ProcID, to the RootProcedureState.
172   * The RootProcedureState contains the execution stack of the Root-Procedure,
173   * It is added to the map by submitProcedure() and removed on procedure completion.
174   */
175  private final ConcurrentHashMap<Long, RootProcedureState<TEnvironment>> rollbackStack =
176    new ConcurrentHashMap<>();
177
178  /**
179   * Helper map to lookup the live procedures by ID.
180   * This map contains every procedure. root-procedures and subprocedures.
181   */
182  private final ConcurrentHashMap<Long, Procedure<TEnvironment>> procedures =
183    new ConcurrentHashMap<>();
184
185  /**
186   * Helper map to lookup whether the procedure already issued from the same client. This map
187   * contains every root procedure.
188   */
189  private final ConcurrentHashMap<NonceKey, Long> nonceKeysToProcIdsMap = new ConcurrentHashMap<>();
190
191  private final CopyOnWriteArrayList<ProcedureExecutorListener> listeners =
192    new CopyOnWriteArrayList<>();
193
194  private Configuration conf;
195
196  /**
197   * Created in the {@link #init(int, boolean)} method. Destroyed in {@link #join()} (FIX! Doing
198   * resource handling rather than observing in a #join is unexpected).
199   * Overridden when we do the ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery
200   * (Should be ok).
201   */
202  private ThreadGroup threadGroup;
203
204  /**
205   * Created in the {@link #init(int, boolean)}  method. Terminated in {@link #join()} (FIX! Doing
206   * resource handling rather than observing in a #join is unexpected).
207   * Overridden when we do the ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery
208   * (Should be ok).
209   */
210  private CopyOnWriteArrayList<WorkerThread> workerThreads;
211
212  /**
213   * Created in the {@link #init(int, boolean)} method. Terminated in {@link #join()} (FIX! Doing
214   * resource handling rather than observing in a #join is unexpected).
215   * Overridden when we do the ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery
216   * (Should be ok).
217   */
218  private TimeoutExecutorThread<TEnvironment> timeoutExecutor;
219
220  /**
221   * WorkerMonitor check for stuck workers and new worker thread when necessary, for example if
222   * there is no worker to assign meta, it will new worker thread for it, so it is very important.
223   * TimeoutExecutor execute many tasks like DeadServerMetricRegionChore RegionInTransitionChore
224   * and so on, some tasks may execute for a long time so will block other tasks like
225   * WorkerMonitor, so use a dedicated thread for executing WorkerMonitor.
226   */
227  private TimeoutExecutorThread<TEnvironment> workerMonitorExecutor;
228
229  private int corePoolSize;
230  private int maxPoolSize;
231
232  private volatile long keepAliveTime;
233
234  /**
235   * Scheduler/Queue that contains runnable procedures.
236   */
237  private final ProcedureScheduler scheduler;
238
239  private final Executor forceUpdateExecutor = Executors.newSingleThreadExecutor(
240    new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Force-Update-PEWorker-%d").build());
241
242  private final AtomicLong lastProcId = new AtomicLong(-1);
243  private final AtomicLong workerId = new AtomicLong(0);
244  private final AtomicInteger activeExecutorCount = new AtomicInteger(0);
245  private final AtomicBoolean running = new AtomicBoolean(false);
246  private final TEnvironment environment;
247  private final ProcedureStore store;
248
249  private final boolean checkOwnerSet;
250
251  // To prevent concurrent execution of the same procedure.
252  // For some rare cases, especially if the procedure uses ProcedureEvent, it is possible that the
253  // procedure is woken up before we finish the suspend which causes the same procedures to be
254  // executed in parallel. This does lead to some problems, see HBASE-20939&HBASE-20949, and is also
255  // a bit confusing to the developers. So here we introduce this lock to prevent the concurrent
256  // execution of the same procedure.
257  private final IdLock procExecutionLock = new IdLock();
258
259  public ProcedureExecutor(final Configuration conf, final TEnvironment environment,
260      final ProcedureStore store) {
261    this(conf, environment, store, new SimpleProcedureScheduler());
262  }
263
264  private boolean isRootFinished(Procedure<?> proc) {
265    Procedure<?> rootProc = procedures.get(proc.getRootProcId());
266    return rootProc == null || rootProc.isFinished();
267  }
268
269  private void forceUpdateProcedure(long procId) throws IOException {
270    IdLock.Entry lockEntry = procExecutionLock.getLockEntry(procId);
271    try {
272      Procedure<TEnvironment> proc = procedures.get(procId);
273      if (proc != null) {
274        if (proc.isFinished() && proc.hasParent() && isRootFinished(proc)) {
275          LOG.debug("Procedure {} has already been finished and parent is succeeded," +
276            " skip force updating", proc);
277          return;
278        }
279      } else {
280        CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
281        if (retainer == null || retainer.getProcedure() instanceof FailedProcedure) {
282          LOG.debug("No pending procedure with id = {}, skip force updating.", procId);
283          return;
284        }
285        long evictTtl = conf.getInt(EVICT_TTL_CONF_KEY, DEFAULT_EVICT_TTL);
286        long evictAckTtl = conf.getInt(EVICT_ACKED_TTL_CONF_KEY, DEFAULT_ACKED_EVICT_TTL);
287        if (retainer.isExpired(System.currentTimeMillis(), evictTtl, evictAckTtl)) {
288          LOG.debug("Procedure {} has already been finished and expired, skip force updating",
289            procId);
290          return;
291        }
292        proc = retainer.getProcedure();
293      }
294      LOG.debug("Force update procedure {}", proc);
295      store.update(proc);
296    } finally {
297      procExecutionLock.releaseLockEntry(lockEntry);
298    }
299  }
300
301  public ProcedureExecutor(final Configuration conf, final TEnvironment environment,
302      final ProcedureStore store, final ProcedureScheduler scheduler) {
303    this.environment = environment;
304    this.scheduler = scheduler;
305    this.store = store;
306    this.conf = conf;
307    this.checkOwnerSet = conf.getBoolean(CHECK_OWNER_SET_CONF_KEY, DEFAULT_CHECK_OWNER_SET);
308    refreshConfiguration(conf);
309    store.registerListener(new ProcedureStoreListener() {
310
311      @Override
312      public void forceUpdate(long[] procIds) {
313        Arrays.stream(procIds).forEach(procId -> forceUpdateExecutor.execute(() -> {
314          try {
315            forceUpdateProcedure(procId);
316          } catch (IOException e) {
317            LOG.warn("Failed to force update procedure with pid={}", procId);
318          }
319        }));
320      }
321    });
322  }
323
324  private void load(final boolean abortOnCorruption) throws IOException {
325    Preconditions.checkArgument(completed.isEmpty(), "completed not empty");
326    Preconditions.checkArgument(rollbackStack.isEmpty(), "rollback state not empty");
327    Preconditions.checkArgument(procedures.isEmpty(), "procedure map not empty");
328    Preconditions.checkArgument(scheduler.size() == 0, "run queue not empty");
329
330    store.load(new ProcedureStore.ProcedureLoader() {
331      @Override
332      public void setMaxProcId(long maxProcId) {
333        assert lastProcId.get() < 0 : "expected only one call to setMaxProcId()";
334        lastProcId.set(maxProcId);
335      }
336
337      @Override
338      public void load(ProcedureIterator procIter) throws IOException {
339        loadProcedures(procIter, abortOnCorruption);
340      }
341
342      @Override
343      public void handleCorrupted(ProcedureIterator procIter) throws IOException {
344        int corruptedCount = 0;
345        while (procIter.hasNext()) {
346          Procedure<?> proc = procIter.next();
347          LOG.error("Corrupt " + proc);
348          corruptedCount++;
349        }
350        if (abortOnCorruption && corruptedCount > 0) {
351          throw new IOException("found " + corruptedCount + " corrupted procedure(s) on replay");
352        }
353      }
354    });
355  }
356
357  private void restoreLock(Procedure<TEnvironment> proc, Set<Long> restored) {
358    proc.restoreLock(getEnvironment());
359    restored.add(proc.getProcId());
360  }
361
362  private void restoreLocks(Deque<Procedure<TEnvironment>> stack, Set<Long> restored) {
363    while (!stack.isEmpty()) {
364      restoreLock(stack.pop(), restored);
365    }
366  }
367
368  // Restore the locks for all the procedures.
369  // Notice that we need to restore the locks starting from the root proc, otherwise there will be
370  // problem that a sub procedure may hold the exclusive lock first and then we are stuck when
371  // calling the acquireLock method for the parent procedure.
372  // The algorithm is straight-forward:
373  // 1. Use a set to record the procedures which locks have already been restored.
374  // 2. Use a stack to store the hierarchy of the procedures
375  // 3. For all the procedure, we will first try to find its parent and push it into the stack,
376  // unless
377  // a. We have no parent, i.e, we are the root procedure
378  // b. The lock has already been restored(by checking the set introduced in #1)
379  // then we start to pop the stack and call acquireLock for each procedure.
380  // Notice that this should be done for all procedures, not only the ones in runnableList.
381  private void restoreLocks() {
382    Set<Long> restored = new HashSet<>();
383    Deque<Procedure<TEnvironment>> stack = new ArrayDeque<>();
384    procedures.values().forEach(proc -> {
385      for (;;) {
386        if (restored.contains(proc.getProcId())) {
387          restoreLocks(stack, restored);
388          return;
389        }
390        if (!proc.hasParent()) {
391          restoreLock(proc, restored);
392          restoreLocks(stack, restored);
393          return;
394        }
395        stack.push(proc);
396        proc = procedures.get(proc.getParentProcId());
397      }
398    });
399  }
400
401  private void loadProcedures(ProcedureIterator procIter, boolean abortOnCorruption)
402      throws IOException {
403    // 1. Build the rollback stack
404    int runnableCount = 0;
405    int failedCount = 0;
406    int waitingCount = 0;
407    int waitingTimeoutCount = 0;
408    while (procIter.hasNext()) {
409      boolean finished = procIter.isNextFinished();
410      @SuppressWarnings("unchecked")
411      Procedure<TEnvironment> proc = procIter.next();
412      NonceKey nonceKey = proc.getNonceKey();
413      long procId = proc.getProcId();
414
415      if (finished) {
416        completed.put(proc.getProcId(), new CompletedProcedureRetainer<>(proc));
417        LOG.debug("Completed {}", proc);
418      } else {
419        if (!proc.hasParent()) {
420          assert !proc.isFinished() : "unexpected finished procedure";
421          rollbackStack.put(proc.getProcId(), new RootProcedureState<>());
422        }
423
424        // add the procedure to the map
425        proc.beforeReplay(getEnvironment());
426        procedures.put(proc.getProcId(), proc);
427        switch (proc.getState()) {
428          case RUNNABLE:
429            runnableCount++;
430            break;
431          case FAILED:
432            failedCount++;
433            break;
434          case WAITING:
435            waitingCount++;
436            break;
437          case WAITING_TIMEOUT:
438            waitingTimeoutCount++;
439            break;
440          default:
441            break;
442        }
443      }
444
445      if (nonceKey != null) {
446        nonceKeysToProcIdsMap.put(nonceKey, procId); // add the nonce to the map
447      }
448    }
449
450    // 2. Initialize the stacks: In the old implementation, for procedures in FAILED state, we will
451    // push it into the ProcedureScheduler directly to execute the rollback. But this does not work
452    // after we introduce the restore lock stage. For now, when we acquire a xlock, we will remove
453    // the queue from runQueue in scheduler, and then when a procedure which has lock access, for
454    // example, a sub procedure of the procedure which has the xlock, is pushed into the scheduler,
455    // we will add the queue back to let the workers poll from it. The assumption here is that, the
456    // procedure which has the xlock should have been polled out already, so when loading we can not
457    // add the procedure to scheduler first and then call acquireLock, since the procedure is still
458    // in the queue, and since we will remove the queue from runQueue, then no one can poll it out,
459    // then there is a dead lock
460    List<Procedure<TEnvironment>> runnableList = new ArrayList<>(runnableCount);
461    List<Procedure<TEnvironment>> failedList = new ArrayList<>(failedCount);
462    List<Procedure<TEnvironment>> waitingList = new ArrayList<>(waitingCount);
463    List<Procedure<TEnvironment>> waitingTimeoutList = new ArrayList<>(waitingTimeoutCount);
464    procIter.reset();
465    while (procIter.hasNext()) {
466      if (procIter.isNextFinished()) {
467        procIter.skipNext();
468        continue;
469      }
470
471      @SuppressWarnings("unchecked")
472      Procedure<TEnvironment> proc = procIter.next();
473      assert !(proc.isFinished() && !proc.hasParent()) : "unexpected completed proc=" + proc;
474      LOG.debug("Loading {}", proc);
475      Long rootProcId = getRootProcedureId(proc);
476      // The orphan procedures will be passed to handleCorrupted, so add an assert here
477      assert rootProcId != null;
478
479      if (proc.hasParent()) {
480        Procedure<TEnvironment> parent = procedures.get(proc.getParentProcId());
481        if (parent != null && !proc.isFinished()) {
482          parent.incChildrenLatch();
483        }
484      }
485
486      RootProcedureState<TEnvironment> procStack = rollbackStack.get(rootProcId);
487      procStack.loadStack(proc);
488
489      proc.setRootProcId(rootProcId);
490      switch (proc.getState()) {
491        case RUNNABLE:
492          runnableList.add(proc);
493          break;
494        case WAITING:
495          waitingList.add(proc);
496          break;
497        case WAITING_TIMEOUT:
498          waitingTimeoutList.add(proc);
499          break;
500        case FAILED:
501          failedList.add(proc);
502          break;
503        case ROLLEDBACK:
504        case INITIALIZING:
505          String msg = "Unexpected " + proc.getState() + " state for " + proc;
506          LOG.error(msg);
507          throw new UnsupportedOperationException(msg);
508        default:
509          break;
510      }
511    }
512
513    // 3. Check the waiting procedures to see if some of them can be added to runnable.
514    waitingList.forEach(proc -> {
515      if (!proc.hasChildren()) {
516        // Normally, WAITING procedures should be waken by its children. But, there is a case that,
517        // all the children are successful and before they can wake up their parent procedure, the
518        // master was killed. So, during recovering the procedures from ProcedureWal, its children
519        // are not loaded because of their SUCCESS state. So we need to continue to run this WAITING
520        // procedure. But before executing, we need to set its state to RUNNABLE, otherwise, a
521        // exception will throw:
522        // Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE,
523        // "NOT RUNNABLE! " + procedure.toString());
524        proc.setState(ProcedureState.RUNNABLE);
525        runnableList.add(proc);
526      } else {
527        proc.afterReplay(getEnvironment());
528      }
529    });
530    // 4. restore locks
531    restoreLocks();
532
533    // 5. Push the procedures to the timeout executor
534    waitingTimeoutList.forEach(proc -> {
535      proc.afterReplay(getEnvironment());
536      timeoutExecutor.add(proc);
537    });
538
539    // 6. Push the procedure to the scheduler
540    failedList.forEach(scheduler::addBack);
541    runnableList.forEach(p -> {
542      p.afterReplay(getEnvironment());
543      if (!p.hasParent()) {
544        sendProcedureLoadedNotification(p.getProcId());
545      }
546      scheduler.addBack(p);
547    });
548    // After all procedures put into the queue, signal the worker threads.
549    // Otherwise, there is a race condition. See HBASE-21364.
550    scheduler.signalAll();
551  }
552
553  /**
554   * Initialize the procedure executor, but do not start workers. We will start them later.
555   * <p/>
556   * It calls ProcedureStore.recoverLease() and ProcedureStore.load() to recover the lease, and
557   * ensure a single executor, and start the procedure replay to resume and recover the previous
558   * pending and in-progress procedures.
559   * @param numThreads number of threads available for procedure execution.
560   * @param abortOnCorruption true if you want to abort your service in case a corrupted procedure
561   *          is found on replay. otherwise false.
562   */
563  public void init(int numThreads, boolean abortOnCorruption) throws IOException {
564    // We have numThreads executor + one timer thread used for timing out
565    // procedures and triggering periodic procedures.
566    this.corePoolSize = numThreads;
567    this.maxPoolSize = 10 * numThreads;
568    LOG.info("Starting {} core workers (bigger of cpus/4 or 16) with max (burst) worker count={}",
569        corePoolSize, maxPoolSize);
570
571    this.threadGroup = new ThreadGroup("PEWorkerGroup");
572    this.timeoutExecutor = new TimeoutExecutorThread<>(this, threadGroup, "ProcExecTimeout");
573    this.workerMonitorExecutor = new TimeoutExecutorThread<>(this, threadGroup, "WorkerMonitor");
574
575    // Create the workers
576    workerId.set(0);
577    workerThreads = new CopyOnWriteArrayList<>();
578    for (int i = 0; i < corePoolSize; ++i) {
579      workerThreads.add(new WorkerThread(threadGroup));
580    }
581
582    long st, et;
583
584    // Acquire the store lease.
585    st = System.nanoTime();
586    store.recoverLease();
587    et = System.nanoTime();
588    LOG.info("Recovered {} lease in {}", store.getClass().getSimpleName(),
589      StringUtils.humanTimeDiff(TimeUnit.NANOSECONDS.toMillis(et - st)));
590
591    // start the procedure scheduler
592    scheduler.start();
593
594    // TODO: Split in two steps.
595    // TODO: Handle corrupted procedures (currently just a warn)
596    // The first one will make sure that we have the latest id,
597    // so we can start the threads and accept new procedures.
598    // The second step will do the actual load of old procedures.
599    st = System.nanoTime();
600    load(abortOnCorruption);
601    et = System.nanoTime();
602    LOG.info("Loaded {} in {}", store.getClass().getSimpleName(),
603      StringUtils.humanTimeDiff(TimeUnit.NANOSECONDS.toMillis(et - st)));
604  }
605
606  /**
607   * Start the workers.
608   */
609  public void startWorkers() throws IOException {
610    if (!running.compareAndSet(false, true)) {
611      LOG.warn("Already running");
612      return;
613    }
614    // Start the executors. Here we must have the lastProcId set.
615    LOG.trace("Start workers {}", workerThreads.size());
616    timeoutExecutor.start();
617    workerMonitorExecutor.start();
618    for (WorkerThread worker: workerThreads) {
619      worker.start();
620    }
621
622    // Internal chores
623    workerMonitorExecutor.add(new WorkerMonitor());
624
625    // Add completed cleaner chore
626    addChore(new CompletedProcedureCleaner<>(conf, store, procExecutionLock, completed,
627      nonceKeysToProcIdsMap));
628  }
629
630  public void stop() {
631    if (!running.getAndSet(false)) {
632      return;
633    }
634
635    LOG.info("Stopping");
636    scheduler.stop();
637    timeoutExecutor.sendStopSignal();
638    workerMonitorExecutor.sendStopSignal();
639  }
640
641  public void join() {
642    assert !isRunning() : "expected not running";
643
644    // stop the timeout executor
645    timeoutExecutor.awaitTermination();
646    // stop the work monitor executor
647    workerMonitorExecutor.awaitTermination();
648
649    // stop the worker threads
650    for (WorkerThread worker: workerThreads) {
651      worker.awaitTermination();
652    }
653
654    // Destroy the Thread Group for the executors
655    // TODO: Fix. #join is not place to destroy resources.
656    try {
657      threadGroup.destroy();
658    } catch (IllegalThreadStateException e) {
659      LOG.error("ThreadGroup {} contains running threads; {}: See STDOUT",
660          this.threadGroup, e.getMessage());
661      // This dumps list of threads on STDOUT.
662      this.threadGroup.list();
663    }
664
665    // reset the in-memory state for testing
666    completed.clear();
667    rollbackStack.clear();
668    procedures.clear();
669    nonceKeysToProcIdsMap.clear();
670    scheduler.clear();
671    lastProcId.set(-1);
672  }
673
674  public void refreshConfiguration(final Configuration conf) {
675    this.conf = conf;
676    setKeepAliveTime(conf.getLong(WORKER_KEEP_ALIVE_TIME_CONF_KEY,
677        DEFAULT_WORKER_KEEP_ALIVE_TIME), TimeUnit.MILLISECONDS);
678  }
679
680  // ==========================================================================
681  //  Accessors
682  // ==========================================================================
683  public boolean isRunning() {
684    return running.get();
685  }
686
687  /**
688   * @return the current number of worker threads.
689   */
690  public int getWorkerThreadCount() {
691    return workerThreads.size();
692  }
693
694  /**
695   * @return the core pool size settings.
696   */
697  public int getCorePoolSize() {
698    return corePoolSize;
699  }
700
701  public int getActiveExecutorCount() {
702    return activeExecutorCount.get();
703  }
704
705  public TEnvironment getEnvironment() {
706    return this.environment;
707  }
708
709  public ProcedureStore getStore() {
710    return this.store;
711  }
712
713  ProcedureScheduler getScheduler() {
714    return scheduler;
715  }
716
717  public void setKeepAliveTime(final long keepAliveTime, final TimeUnit timeUnit) {
718    this.keepAliveTime = timeUnit.toMillis(keepAliveTime);
719    this.scheduler.signalAll();
720  }
721
722  public long getKeepAliveTime(final TimeUnit timeUnit) {
723    return timeUnit.convert(keepAliveTime, TimeUnit.MILLISECONDS);
724  }
725
726  // ==========================================================================
727  //  Submit/Remove Chores
728  // ==========================================================================
729
730  /**
731   * Add a chore procedure to the executor
732   * @param chore the chore to add
733   */
734  public void addChore(@Nullable ProcedureInMemoryChore<TEnvironment> chore) {
735    if (chore == null) {
736      return;
737    }
738    chore.setState(ProcedureState.WAITING_TIMEOUT);
739    timeoutExecutor.add(chore);
740  }
741
742  /**
743   * Remove a chore procedure from the executor
744   * @param chore the chore to remove
745   * @return whether the chore is removed, or it will be removed later
746   */
747  public boolean removeChore(@Nullable ProcedureInMemoryChore<TEnvironment> chore) {
748    if (chore == null) {
749      return true;
750    }
751    chore.setState(ProcedureState.SUCCESS);
752    return timeoutExecutor.remove(chore);
753  }
754
755  // ==========================================================================
756  //  Nonce Procedure helpers
757  // ==========================================================================
758  /**
759   * Create a NonceKey from the specified nonceGroup and nonce.
760   * @param nonceGroup the group to use for the {@link NonceKey}
761   * @param nonce the nonce to use in the {@link NonceKey}
762   * @return the generated NonceKey
763   */
764  public NonceKey createNonceKey(final long nonceGroup, final long nonce) {
765    return (nonce == HConstants.NO_NONCE) ? null : new NonceKey(nonceGroup, nonce);
766  }
767
768  /**
769   * Register a nonce for a procedure that is going to be submitted.
770   * A procId will be reserved and on submitProcedure(),
771   * the procedure with the specified nonce will take the reserved ProcId.
772   * If someone already reserved the nonce, this method will return the procId reserved,
773   * otherwise an invalid procId will be returned. and the caller should procede
774   * and submit the procedure.
775   *
776   * @param nonceKey A unique identifier for this operation from the client or process.
777   * @return the procId associated with the nonce, if any otherwise an invalid procId.
778   */
779  public long registerNonce(final NonceKey nonceKey) {
780    if (nonceKey == null) {
781      return -1;
782    }
783
784    // check if we have already a Reserved ID for the nonce
785    Long oldProcId = nonceKeysToProcIdsMap.get(nonceKey);
786    if (oldProcId == null) {
787      // reserve a new Procedure ID, this will be associated with the nonce
788      // and the procedure submitted with the specified nonce will use this ID.
789      final long newProcId = nextProcId();
790      oldProcId = nonceKeysToProcIdsMap.putIfAbsent(nonceKey, newProcId);
791      if (oldProcId == null) {
792        return -1;
793      }
794    }
795
796    // we found a registered nonce, but the procedure may not have been submitted yet.
797    // since the client expect the procedure to be submitted, spin here until it is.
798    final boolean traceEnabled = LOG.isTraceEnabled();
799    while (isRunning() &&
800           !(procedures.containsKey(oldProcId) || completed.containsKey(oldProcId)) &&
801           nonceKeysToProcIdsMap.containsKey(nonceKey)) {
802      if (traceEnabled) {
803        LOG.trace("Waiting for pid=" + oldProcId.longValue() + " to be submitted");
804      }
805      Threads.sleep(100);
806    }
807    return oldProcId.longValue();
808  }
809
810  /**
811   * Remove the NonceKey if the procedure was not submitted to the executor.
812   * @param nonceKey A unique identifier for this operation from the client or process.
813   */
814  public void unregisterNonceIfProcedureWasNotSubmitted(final NonceKey nonceKey) {
815    if (nonceKey == null) {
816      return;
817    }
818
819    final Long procId = nonceKeysToProcIdsMap.get(nonceKey);
820    if (procId == null) {
821      return;
822    }
823
824    // if the procedure was not submitted, remove the nonce
825    if (!(procedures.containsKey(procId) || completed.containsKey(procId))) {
826      nonceKeysToProcIdsMap.remove(nonceKey);
827    }
828  }
829
830  /**
831   * If the failure failed before submitting it, we may want to give back the
832   * same error to the requests with the same nonceKey.
833   *
834   * @param nonceKey A unique identifier for this operation from the client or process
835   * @param procName name of the procedure, used to inform the user
836   * @param procOwner name of the owner of the procedure, used to inform the user
837   * @param exception the failure to report to the user
838   */
839  public void setFailureResultForNonce(NonceKey nonceKey, String procName, User procOwner,
840      IOException exception) {
841    if (nonceKey == null) {
842      return;
843    }
844
845    Long procId = nonceKeysToProcIdsMap.get(nonceKey);
846    if (procId == null || completed.containsKey(procId)) {
847      return;
848    }
849
850    Procedure<TEnvironment> proc =
851      new FailedProcedure<>(procId.longValue(), procName, procOwner, nonceKey, exception);
852
853    completed.putIfAbsent(procId, new CompletedProcedureRetainer<>(proc));
854  }
855
856  // ==========================================================================
857  //  Submit/Abort Procedure
858  // ==========================================================================
859  /**
860   * Add a new root-procedure to the executor.
861   * @param proc the new procedure to execute.
862   * @return the procedure id, that can be used to monitor the operation
863   */
864  public long submitProcedure(Procedure<TEnvironment> proc) {
865    return submitProcedure(proc, null);
866  }
867
868  /**
869   * Bypass a procedure. If the procedure is set to bypass, all the logic in
870   * execute/rollback will be ignored and it will return success, whatever.
871   * It is used to recover buggy stuck procedures, releasing the lock resources
872   * and letting other procedures run. Bypassing one procedure (and its ancestors will
873   * be bypassed automatically) may leave the cluster in a middle state, e.g. region
874   * not assigned, or some hdfs files left behind. After getting rid of those stuck procedures,
875   * the operators may have to do some clean up on hdfs or schedule some assign procedures
876   * to let region online. DO AT YOUR OWN RISK.
877   * <p>
878   * A procedure can be bypassed only if
879   * 1. The procedure is in state of RUNNABLE, WAITING, WAITING_TIMEOUT
880   * or it is a root procedure without any child.
881   * 2. No other worker thread is executing it
882   * 3. No child procedure has been submitted
883   *
884   * <p>
885   * If all the requirements are meet, the procedure and its ancestors will be
886   * bypassed and persisted to WAL.
887   *
888   * <p>
889   * If the procedure is in WAITING state, will set it to RUNNABLE add it to run queue.
890   * TODO: What about WAITING_TIMEOUT?
891   * @param pids the procedure id
892   * @param lockWait time to wait lock
893   * @param force if force set to true, we will bypass the procedure even if it is executing.
894   *              This is for procedures which can't break out during executing(due to bug, mostly)
895   *              In this case, bypassing the procedure is not enough, since it is already stuck
896   *              there. We need to restart the master after bypassing, and letting the problematic
897   *              procedure to execute wth bypass=true, so in that condition, the procedure can be
898   *              successfully bypassed.
899   * @param recursive We will do an expensive search for children of each pid. EXPENSIVE!
900   * @return true if bypass success
901   * @throws IOException IOException
902   */
903  public List<Boolean> bypassProcedure(List<Long> pids, long lockWait, boolean force,
904      boolean recursive)
905      throws IOException {
906    List<Boolean> result = new ArrayList<Boolean>(pids.size());
907    for(long pid: pids) {
908      result.add(bypassProcedure(pid, lockWait, force, recursive));
909    }
910    return result;
911  }
912
913  boolean bypassProcedure(long pid, long lockWait, boolean override, boolean recursive)
914      throws IOException {
915    Preconditions.checkArgument(lockWait > 0, "lockWait should be positive");
916    final Procedure<TEnvironment> procedure = getProcedure(pid);
917    if (procedure == null) {
918      LOG.debug("Procedure pid={} does not exist, skipping bypass", pid);
919      return false;
920    }
921
922    LOG.debug("Begin bypass {} with lockWait={}, override={}, recursive={}",
923        procedure, lockWait, override, recursive);
924    IdLock.Entry lockEntry = procExecutionLock.tryLockEntry(procedure.getProcId(), lockWait);
925    if (lockEntry == null && !override) {
926      LOG.debug("Waited {} ms, but {} is still running, skipping bypass with force={}",
927          lockWait, procedure, override);
928      return false;
929    } else if (lockEntry == null) {
930      LOG.debug("Waited {} ms, but {} is still running, begin bypass with force={}",
931          lockWait, procedure, override);
932    }
933    try {
934      // check whether the procedure is already finished
935      if (procedure.isFinished()) {
936        LOG.debug("{} is already finished, skipping bypass", procedure);
937        return false;
938      }
939
940      if (procedure.hasChildren()) {
941        if (recursive) {
942          // EXPENSIVE. Checks each live procedure of which there could be many!!!
943          // Is there another way to get children of a procedure?
944          LOG.info("Recursive bypass on children of pid={}", procedure.getProcId());
945          this.procedures.forEachValue(1 /*Single-threaded*/,
946            // Transformer
947            v -> v.getParentProcId() == procedure.getProcId()? v: null,
948            // Consumer
949            v -> {
950              try {
951                bypassProcedure(v.getProcId(), lockWait, override, recursive);
952              } catch (IOException e) {
953                LOG.warn("Recursive bypass of pid={}", v.getProcId(), e);
954              }
955            });
956        } else {
957          LOG.debug("{} has children, skipping bypass", procedure);
958          return false;
959        }
960      }
961
962      // If the procedure has no parent or no child, we are safe to bypass it in whatever state
963      if (procedure.hasParent() && procedure.getState() != ProcedureState.RUNNABLE
964          && procedure.getState() != ProcedureState.WAITING
965          && procedure.getState() != ProcedureState.WAITING_TIMEOUT) {
966        LOG.debug("Bypassing procedures in RUNNABLE, WAITING and WAITING_TIMEOUT states "
967                + "(with no parent), {}",
968            procedure);
969        // Question: how is the bypass done here?
970        return false;
971      }
972
973      // Now, the procedure is not finished, and no one can execute it since we take the lock now
974      // And we can be sure that its ancestor is not running too, since their child has not
975      // finished yet
976      Procedure<TEnvironment> current = procedure;
977      while (current != null) {
978        LOG.debug("Bypassing {}", current);
979        current.bypass(getEnvironment());
980        store.update(current);
981        long parentID = current.getParentProcId();
982        current = getProcedure(parentID);
983      }
984
985      //wake up waiting procedure, already checked there is no child
986      if (procedure.getState() == ProcedureState.WAITING) {
987        procedure.setState(ProcedureState.RUNNABLE);
988        store.update(procedure);
989      }
990
991      // If state of procedure is WAITING_TIMEOUT, we can directly submit it to the scheduler.
992      // Instead we should remove it from timeout Executor queue and tranfer its state to RUNNABLE
993      if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
994        LOG.debug("transform procedure {} from WAITING_TIMEOUT to RUNNABLE", procedure);
995        if (timeoutExecutor.remove(procedure)) {
996          LOG.debug("removed procedure {} from timeoutExecutor", procedure);
997          timeoutExecutor.executeTimedoutProcedure(procedure);
998        }
999      } else if (lockEntry != null) {
1000        scheduler.addFront(procedure);
1001        LOG.debug("Bypassing {} and its ancestors successfully, adding to queue", procedure);
1002      } else {
1003        // If we don't have the lock, we can't re-submit the queue,
1004        // since it is already executing. To get rid of the stuck situation, we
1005        // need to restart the master. With the procedure set to bypass, the procedureExecutor
1006        // will bypass it and won't get stuck again.
1007        LOG.debug("Bypassing {} and its ancestors successfully, but since it is already running, "
1008            + "skipping add to queue",
1009          procedure);
1010      }
1011      return true;
1012
1013    } finally {
1014      if (lockEntry != null) {
1015        procExecutionLock.releaseLockEntry(lockEntry);
1016      }
1017    }
1018  }
1019
1020  /**
1021   * Add a new root-procedure to the executor.
1022   * @param proc the new procedure to execute.
1023   * @param nonceKey the registered unique identifier for this operation from the client or process.
1024   * @return the procedure id, that can be used to monitor the operation
1025   */
1026  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
1027      justification = "FindBugs is blind to the check-for-null")
1028  public long submitProcedure(Procedure<TEnvironment> proc, NonceKey nonceKey) {
1029    Preconditions.checkArgument(lastProcId.get() >= 0);
1030
1031    prepareProcedure(proc);
1032
1033    final Long currentProcId;
1034    if (nonceKey != null) {
1035      currentProcId = nonceKeysToProcIdsMap.get(nonceKey);
1036      Preconditions.checkArgument(currentProcId != null,
1037        "Expected nonceKey=" + nonceKey + " to be reserved, use registerNonce(); proc=" + proc);
1038    } else {
1039      currentProcId = nextProcId();
1040    }
1041
1042    // Initialize the procedure
1043    proc.setNonceKey(nonceKey);
1044    proc.setProcId(currentProcId.longValue());
1045
1046    // Commit the transaction
1047    store.insert(proc, null);
1048    LOG.debug("Stored {}", proc);
1049
1050    // Add the procedure to the executor
1051    return pushProcedure(proc);
1052  }
1053
1054  /**
1055   * Add a set of new root-procedure to the executor.
1056   * @param procs the new procedures to execute.
1057   */
1058  // TODO: Do we need to take nonces here?
1059  public void submitProcedures(Procedure<TEnvironment>[] procs) {
1060    Preconditions.checkArgument(lastProcId.get() >= 0);
1061    if (procs == null || procs.length <= 0) {
1062      return;
1063    }
1064
1065    // Prepare procedure
1066    for (int i = 0; i < procs.length; ++i) {
1067      prepareProcedure(procs[i]).setProcId(nextProcId());
1068    }
1069
1070    // Commit the transaction
1071    store.insert(procs);
1072    if (LOG.isDebugEnabled()) {
1073      LOG.debug("Stored " + Arrays.toString(procs));
1074    }
1075
1076    // Add the procedure to the executor
1077    for (int i = 0; i < procs.length; ++i) {
1078      pushProcedure(procs[i]);
1079    }
1080  }
1081
1082  private Procedure<TEnvironment> prepareProcedure(Procedure<TEnvironment> proc) {
1083    Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING);
1084    Preconditions.checkArgument(!proc.hasParent(), "unexpected parent", proc);
1085    if (this.checkOwnerSet) {
1086      Preconditions.checkArgument(proc.hasOwner(), "missing owner");
1087    }
1088    return proc;
1089  }
1090
1091  private long pushProcedure(Procedure<TEnvironment> proc) {
1092    final long currentProcId = proc.getProcId();
1093
1094    // Update metrics on start of a procedure
1095    proc.updateMetricsOnSubmit(getEnvironment());
1096
1097    // Create the rollback stack for the procedure
1098    RootProcedureState<TEnvironment> stack = new RootProcedureState<>();
1099    rollbackStack.put(currentProcId, stack);
1100
1101    // Submit the new subprocedures
1102    assert !procedures.containsKey(currentProcId);
1103    procedures.put(currentProcId, proc);
1104    sendProcedureAddedNotification(currentProcId);
1105    scheduler.addBack(proc);
1106    return proc.getProcId();
1107  }
1108
1109  /**
1110   * Send an abort notification the specified procedure.
1111   * Depending on the procedure implementation the abort can be considered or ignored.
1112   * @param procId the procedure to abort
1113   * @return true if the procedure exists and has received the abort, otherwise false.
1114   */
1115  public boolean abort(long procId) {
1116    return abort(procId, true);
1117  }
1118
1119  /**
1120   * Send an abort notification to the specified procedure.
1121   * Depending on the procedure implementation, the abort can be considered or ignored.
1122   * @param procId the procedure to abort
1123   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
1124   * @return true if the procedure exists and has received the abort, otherwise false.
1125   */
1126  public boolean abort(long procId, boolean mayInterruptIfRunning) {
1127    Procedure<TEnvironment> proc = procedures.get(procId);
1128    if (proc != null) {
1129      if (!mayInterruptIfRunning && proc.wasExecuted()) {
1130        return false;
1131      }
1132      return proc.abort(getEnvironment());
1133    }
1134    return false;
1135  }
1136
1137  // ==========================================================================
1138  //  Executor query helpers
1139  // ==========================================================================
1140  public Procedure<TEnvironment> getProcedure(final long procId) {
1141    return procedures.get(procId);
1142  }
1143
1144  public <T extends Procedure<TEnvironment>> T getProcedure(Class<T> clazz, long procId) {
1145    Procedure<TEnvironment> proc = getProcedure(procId);
1146    if (clazz.isInstance(proc)) {
1147      return clazz.cast(proc);
1148    }
1149    return null;
1150  }
1151
1152  public Procedure<TEnvironment> getResult(long procId) {
1153    CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
1154    if (retainer == null) {
1155      return null;
1156    } else {
1157      return retainer.getProcedure();
1158    }
1159  }
1160
1161  /**
1162   * Return true if the procedure is finished.
1163   * The state may be "completed successfully" or "failed and rolledback".
1164   * Use getResult() to check the state or get the result data.
1165   * @param procId the ID of the procedure to check
1166   * @return true if the procedure execution is finished, otherwise false.
1167   */
1168  public boolean isFinished(final long procId) {
1169    return !procedures.containsKey(procId);
1170  }
1171
1172  /**
1173   * Return true if the procedure is started.
1174   * @param procId the ID of the procedure to check
1175   * @return true if the procedure execution is started, otherwise false.
1176   */
1177  public boolean isStarted(long procId) {
1178    Procedure<?> proc = procedures.get(procId);
1179    if (proc == null) {
1180      return completed.get(procId) != null;
1181    }
1182    return proc.wasExecuted();
1183  }
1184
1185  /**
1186   * Mark the specified completed procedure, as ready to remove.
1187   * @param procId the ID of the procedure to remove
1188   */
1189  public void removeResult(long procId) {
1190    CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
1191    if (retainer == null) {
1192      assert !procedures.containsKey(procId) : "pid=" + procId + " is still running";
1193      LOG.debug("pid={} already removed by the cleaner.", procId);
1194      return;
1195    }
1196
1197    // The CompletedProcedureCleaner will take care of deletion, once the TTL is expired.
1198    retainer.setClientAckTime(EnvironmentEdgeManager.currentTime());
1199  }
1200
1201  public Procedure<TEnvironment> getResultOrProcedure(long procId) {
1202    CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
1203    if (retainer == null) {
1204      return procedures.get(procId);
1205    } else {
1206      return retainer.getProcedure();
1207    }
1208  }
1209
1210  /**
1211   * Check if the user is this procedure's owner
1212   * @param procId the target procedure
1213   * @param user the user
1214   * @return true if the user is the owner of the procedure,
1215   *   false otherwise or the owner is unknown.
1216   */
1217  public boolean isProcedureOwner(long procId, User user) {
1218    if (user == null) {
1219      return false;
1220    }
1221    final Procedure<TEnvironment> runningProc = procedures.get(procId);
1222    if (runningProc != null) {
1223      return runningProc.getOwner().equals(user.getShortName());
1224    }
1225
1226    final CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
1227    if (retainer != null) {
1228      return retainer.getProcedure().getOwner().equals(user.getShortName());
1229    }
1230
1231    // Procedure either does not exist or has already completed and got cleaned up.
1232    // At this time, we cannot check the owner of the procedure
1233    return false;
1234  }
1235
1236  /**
1237   * Should only be used when starting up, where the procedure workers have not been started.
1238   * <p/>
1239   * If the procedure works has been started, the return values maybe changed when you are
1240   * processing it so usually this is not safe. Use {@link #getProcedures()} below for most cases as
1241   * it will do a copy, and also include the finished procedures.
1242   */
1243  public Collection<Procedure<TEnvironment>> getActiveProceduresNoCopy() {
1244    return procedures.values();
1245  }
1246
1247  /**
1248   * Get procedures.
1249   * @return the procedures in a list
1250   */
1251  public List<Procedure<TEnvironment>> getProcedures() {
1252    List<Procedure<TEnvironment>> procedureList =
1253      new ArrayList<>(procedures.size() + completed.size());
1254    procedureList.addAll(procedures.values());
1255    // Note: The procedure could show up twice in the list with different state, as
1256    // it could complete after we walk through procedures list and insert into
1257    // procedureList - it is ok, as we will use the information in the Procedure
1258    // to figure it out; to prevent this would increase the complexity of the logic.
1259    completed.values().stream().map(CompletedProcedureRetainer::getProcedure)
1260      .forEach(procedureList::add);
1261    return procedureList;
1262  }
1263
1264  // ==========================================================================
1265  //  Listeners helpers
1266  // ==========================================================================
1267  public void registerListener(ProcedureExecutorListener listener) {
1268    this.listeners.add(listener);
1269  }
1270
1271  public boolean unregisterListener(ProcedureExecutorListener listener) {
1272    return this.listeners.remove(listener);
1273  }
1274
1275  private void sendProcedureLoadedNotification(final long procId) {
1276    if (!this.listeners.isEmpty()) {
1277      for (ProcedureExecutorListener listener: this.listeners) {
1278        try {
1279          listener.procedureLoaded(procId);
1280        } catch (Throwable e) {
1281          LOG.error("Listener " + listener + " had an error: " + e.getMessage(), e);
1282        }
1283      }
1284    }
1285  }
1286
1287  private void sendProcedureAddedNotification(final long procId) {
1288    if (!this.listeners.isEmpty()) {
1289      for (ProcedureExecutorListener listener: this.listeners) {
1290        try {
1291          listener.procedureAdded(procId);
1292        } catch (Throwable e) {
1293          LOG.error("Listener " + listener + " had an error: " + e.getMessage(), e);
1294        }
1295      }
1296    }
1297  }
1298
1299  private void sendProcedureFinishedNotification(final long procId) {
1300    if (!this.listeners.isEmpty()) {
1301      for (ProcedureExecutorListener listener: this.listeners) {
1302        try {
1303          listener.procedureFinished(procId);
1304        } catch (Throwable e) {
1305          LOG.error("Listener " + listener + " had an error: " + e.getMessage(), e);
1306        }
1307      }
1308    }
1309  }
1310
1311  // ==========================================================================
1312  //  Procedure IDs helpers
1313  // ==========================================================================
1314  private long nextProcId() {
1315    long procId = lastProcId.incrementAndGet();
1316    if (procId < 0) {
1317      while (!lastProcId.compareAndSet(procId, 0)) {
1318        procId = lastProcId.get();
1319        if (procId >= 0) {
1320          break;
1321        }
1322      }
1323      while (procedures.containsKey(procId)) {
1324        procId = lastProcId.incrementAndGet();
1325      }
1326    }
1327    assert procId >= 0 : "Invalid procId " + procId;
1328    return procId;
1329  }
1330
1331  protected long getLastProcId() {
1332    return lastProcId.get();
1333  }
1334
1335  public Set<Long> getActiveProcIds() {
1336    return procedures.keySet();
1337  }
1338
1339  Long getRootProcedureId(Procedure<TEnvironment> proc) {
1340    return Procedure.getRootProcedureId(procedures, proc);
1341  }
1342
1343  // ==========================================================================
1344  //  Executions
1345  // ==========================================================================
1346  private void executeProcedure(Procedure<TEnvironment> proc) {
1347    if (proc.isFinished()) {
1348      LOG.debug("{} is already finished, skipping execution", proc);
1349      return;
1350    }
1351    final Long rootProcId = getRootProcedureId(proc);
1352    if (rootProcId == null) {
1353      // The 'proc' was ready to run but the root procedure was rolledback
1354      LOG.warn("Rollback because parent is done/rolledback proc=" + proc);
1355      executeRollback(proc);
1356      return;
1357    }
1358
1359    RootProcedureState<TEnvironment> procStack = rollbackStack.get(rootProcId);
1360    if (procStack == null) {
1361      LOG.warn("RootProcedureState is null for " + proc.getProcId());
1362      return;
1363    }
1364    do {
1365      // Try to acquire the execution
1366      if (!procStack.acquire(proc)) {
1367        if (procStack.setRollback()) {
1368          // we have the 'rollback-lock' we can start rollingback
1369          switch (executeRollback(rootProcId, procStack)) {
1370            case LOCK_ACQUIRED:
1371              break;
1372            case LOCK_YIELD_WAIT:
1373              procStack.unsetRollback();
1374              scheduler.yield(proc);
1375              break;
1376            case LOCK_EVENT_WAIT:
1377              LOG.info("LOCK_EVENT_WAIT rollback..." + proc);
1378              procStack.unsetRollback();
1379              break;
1380            default:
1381              throw new UnsupportedOperationException();
1382          }
1383        } else {
1384          // if we can't rollback means that some child is still running.
1385          // the rollback will be executed after all the children are done.
1386          // If the procedure was never executed, remove and mark it as rolledback.
1387          if (!proc.wasExecuted()) {
1388            switch (executeRollback(proc)) {
1389              case LOCK_ACQUIRED:
1390                break;
1391              case LOCK_YIELD_WAIT:
1392                scheduler.yield(proc);
1393                break;
1394              case LOCK_EVENT_WAIT:
1395                LOG.info("LOCK_EVENT_WAIT can't rollback child running?..." + proc);
1396                break;
1397              default:
1398                throw new UnsupportedOperationException();
1399            }
1400          }
1401        }
1402        break;
1403      }
1404
1405      // Execute the procedure
1406      assert proc.getState() == ProcedureState.RUNNABLE : proc;
1407      // Note that lock is NOT about concurrency but rather about ensuring
1408      // ownership of a procedure of an entity such as a region or table
1409      LockState lockState = acquireLock(proc);
1410      switch (lockState) {
1411        case LOCK_ACQUIRED:
1412          execProcedure(procStack, proc);
1413          break;
1414        case LOCK_YIELD_WAIT:
1415          LOG.info(lockState + " " + proc);
1416          scheduler.yield(proc);
1417          break;
1418        case LOCK_EVENT_WAIT:
1419          // Someone will wake us up when the lock is available
1420          LOG.debug(lockState + " " + proc);
1421          break;
1422        default:
1423          throw new UnsupportedOperationException();
1424      }
1425      procStack.release(proc);
1426
1427      if (proc.isSuccess()) {
1428        // update metrics on finishing the procedure
1429        proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), true);
1430        LOG.info("Finished " + proc + " in " + StringUtils.humanTimeDiff(proc.elapsedTime()));
1431        // Finalize the procedure state
1432        if (proc.getProcId() == rootProcId) {
1433          procedureFinished(proc);
1434        } else {
1435          execCompletionCleanup(proc);
1436        }
1437        break;
1438      }
1439    } while (procStack.isFailed());
1440  }
1441
1442  private LockState acquireLock(Procedure<TEnvironment> proc) {
1443    TEnvironment env = getEnvironment();
1444    // if holdLock is true, then maybe we already have the lock, so just return LOCK_ACQUIRED if
1445    // hasLock is true.
1446    if (proc.hasLock()) {
1447      return LockState.LOCK_ACQUIRED;
1448    }
1449    return proc.doAcquireLock(env, store);
1450  }
1451
1452  private void releaseLock(Procedure<TEnvironment> proc, boolean force) {
1453    TEnvironment env = getEnvironment();
1454    // For how the framework works, we know that we will always have the lock
1455    // when we call releaseLock(), so we can avoid calling proc.hasLock()
1456    if (force || !proc.holdLock(env) || proc.isFinished()) {
1457      proc.doReleaseLock(env, store);
1458    }
1459  }
1460
1461  /**
1462   * Execute the rollback of the full procedure stack. Once the procedure is rolledback, the
1463   * root-procedure will be visible as finished to user, and the result will be the fatal exception.
1464   */
1465  private LockState executeRollback(long rootProcId, RootProcedureState<TEnvironment> procStack) {
1466    Procedure<TEnvironment> rootProc = procedures.get(rootProcId);
1467    RemoteProcedureException exception = rootProc.getException();
1468    // TODO: This needs doc. The root proc doesn't have an exception. Maybe we are
1469    // rolling back because the subprocedure does. Clarify.
1470    if (exception == null) {
1471      exception = procStack.getException();
1472      rootProc.setFailure(exception);
1473      store.update(rootProc);
1474    }
1475
1476    List<Procedure<TEnvironment>> subprocStack = procStack.getSubproceduresStack();
1477    assert subprocStack != null : "Called rollback with no steps executed rootProc=" + rootProc;
1478
1479    int stackTail = subprocStack.size();
1480    while (stackTail-- > 0) {
1481      Procedure<TEnvironment> proc = subprocStack.get(stackTail);
1482      IdLock.Entry lockEntry = null;
1483      // Hold the execution lock if it is not held by us. The IdLock is not reentrant so we need
1484      // this check, as the worker will hold the lock before executing a procedure. This is the only
1485      // place where we may hold two procedure execution locks, and there is a fence in the
1486      // RootProcedureState where we can make sure that only one worker can execute the rollback of
1487      // a RootProcedureState, so there is no dead lock problem. And the lock here is necessary to
1488      // prevent race between us and the force update thread.
1489      if (!procExecutionLock.isHeldByCurrentThread(proc.getProcId())) {
1490        try {
1491          lockEntry = procExecutionLock.getLockEntry(proc.getProcId());
1492        } catch (IOException e) {
1493          // can only happen if interrupted, so not a big deal to propagate it
1494          throw new UncheckedIOException(e);
1495        }
1496      }
1497      try {
1498        // For the sub procedures which are successfully finished, we do not rollback them.
1499        // Typically, if we want to rollback a procedure, we first need to rollback it, and then
1500        // recursively rollback its ancestors. The state changes which are done by sub procedures
1501        // should be handled by parent procedures when rolling back. For example, when rolling back
1502        // a MergeTableProcedure, we will schedule new procedures to bring the offline regions
1503        // online, instead of rolling back the original procedures which offlined the regions(in
1504        // fact these procedures can not be rolled back...).
1505        if (proc.isSuccess()) {
1506          // Just do the cleanup work, without actually executing the rollback
1507          subprocStack.remove(stackTail);
1508          cleanupAfterRollbackOneStep(proc);
1509          continue;
1510        }
1511        LockState lockState = acquireLock(proc);
1512        if (lockState != LockState.LOCK_ACQUIRED) {
1513          // can't take a lock on the procedure, add the root-proc back on the
1514          // queue waiting for the lock availability
1515          return lockState;
1516        }
1517
1518        lockState = executeRollback(proc);
1519        releaseLock(proc, false);
1520        boolean abortRollback = lockState != LockState.LOCK_ACQUIRED;
1521        abortRollback |= !isRunning() || !store.isRunning();
1522
1523        // allows to kill the executor before something is stored to the wal.
1524        // useful to test the procedure recovery.
1525        if (abortRollback) {
1526          return lockState;
1527        }
1528
1529        subprocStack.remove(stackTail);
1530
1531        // if the procedure is kind enough to pass the slot to someone else, yield
1532        // if the proc is already finished, do not yield
1533        if (!proc.isFinished() && proc.isYieldAfterExecutionStep(getEnvironment())) {
1534          return LockState.LOCK_YIELD_WAIT;
1535        }
1536
1537        if (proc != rootProc) {
1538          execCompletionCleanup(proc);
1539        }
1540      } finally {
1541        if (lockEntry != null) {
1542          procExecutionLock.releaseLockEntry(lockEntry);
1543        }
1544      }
1545    }
1546
1547    // Finalize the procedure state
1548    LOG.info("Rolled back {} exec-time={}", rootProc,
1549      StringUtils.humanTimeDiff(rootProc.elapsedTime()));
1550    procedureFinished(rootProc);
1551    return LockState.LOCK_ACQUIRED;
1552  }
1553
1554  private void cleanupAfterRollbackOneStep(Procedure<TEnvironment> proc) {
1555    if (proc.removeStackIndex()) {
1556      if (!proc.isSuccess()) {
1557        proc.setState(ProcedureState.ROLLEDBACK);
1558      }
1559
1560      // update metrics on finishing the procedure (fail)
1561      proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), false);
1562
1563      if (proc.hasParent()) {
1564        store.delete(proc.getProcId());
1565        procedures.remove(proc.getProcId());
1566      } else {
1567        final long[] childProcIds = rollbackStack.get(proc.getProcId()).getSubprocedureIds();
1568        if (childProcIds != null) {
1569          store.delete(proc, childProcIds);
1570        } else {
1571          store.update(proc);
1572        }
1573      }
1574    } else {
1575      store.update(proc);
1576    }
1577  }
1578
1579  /**
1580   * Execute the rollback of the procedure step.
1581   * It updates the store with the new state (stack index)
1582   * or will remove completly the procedure in case it is a child.
1583   */
1584  private LockState executeRollback(Procedure<TEnvironment> proc) {
1585    try {
1586      proc.doRollback(getEnvironment());
1587    } catch (IOException e) {
1588      LOG.debug("Roll back attempt failed for {}", proc, e);
1589      return LockState.LOCK_YIELD_WAIT;
1590    } catch (InterruptedException e) {
1591      handleInterruptedException(proc, e);
1592      return LockState.LOCK_YIELD_WAIT;
1593    } catch (Throwable e) {
1594      // Catch NullPointerExceptions or similar errors...
1595      LOG.error(HBaseMarkers.FATAL, "CODE-BUG: Uncaught runtime exception for " + proc, e);
1596    }
1597
1598    // allows to kill the executor before something is stored to the wal.
1599    // useful to test the procedure recovery.
1600    if (testing != null && testing.shouldKillBeforeStoreUpdate()) {
1601      String msg = "TESTING: Kill before store update";
1602      LOG.debug(msg);
1603      stop();
1604      throw new RuntimeException(msg);
1605    }
1606
1607    cleanupAfterRollbackOneStep(proc);
1608
1609    return LockState.LOCK_ACQUIRED;
1610  }
1611
1612  private void yieldProcedure(Procedure<TEnvironment> proc) {
1613    releaseLock(proc, false);
1614    scheduler.yield(proc);
1615  }
1616
1617  /**
1618   * Executes <code>procedure</code>
1619   * <ul>
1620   *  <li>Calls the doExecute() of the procedure
1621   *  <li>If the procedure execution didn't fail (i.e. valid user input)
1622   *  <ul>
1623   *    <li>...and returned subprocedures
1624   *    <ul><li>The subprocedures are initialized.
1625   *      <li>The subprocedures are added to the store
1626   *      <li>The subprocedures are added to the runnable queue
1627   *      <li>The procedure is now in a WAITING state, waiting for the subprocedures to complete
1628   *    </ul>
1629   *    </li>
1630   *   <li>...if there are no subprocedure
1631   *    <ul><li>the procedure completed successfully
1632   *      <li>if there is a parent (WAITING)
1633   *      <li>the parent state will be set to RUNNABLE
1634   *    </ul>
1635   *   </li>
1636   *  </ul>
1637   *  </li>
1638   *  <li>In case of failure
1639   *  <ul>
1640   *    <li>The store is updated with the new state</li>
1641   *    <li>The executor (caller of this method) will start the rollback of the procedure</li>
1642   *  </ul>
1643   *  </li>
1644   *  </ul>
1645   */
1646  private void execProcedure(RootProcedureState<TEnvironment> procStack,
1647      Procedure<TEnvironment> procedure) {
1648    Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE,
1649        "NOT RUNNABLE! " + procedure.toString());
1650
1651    // Procedures can suspend themselves. They skip out by throwing a ProcedureSuspendedException.
1652    // The exception is caught below and then we hurry to the exit without disturbing state. The
1653    // idea is that the processing of this procedure will be unsuspended later by an external event
1654    // such the report of a region open.
1655    boolean suspended = false;
1656
1657    // Whether to 're-' -execute; run through the loop again.
1658    boolean reExecute = false;
1659
1660    Procedure<TEnvironment>[] subprocs = null;
1661    do {
1662      reExecute = false;
1663      procedure.resetPersistence();
1664      try {
1665        subprocs = procedure.doExecute(getEnvironment());
1666        if (subprocs != null && subprocs.length == 0) {
1667          subprocs = null;
1668        }
1669      } catch (ProcedureSuspendedException e) {
1670        LOG.trace("Suspend {}", procedure);
1671        suspended = true;
1672      } catch (ProcedureYieldException e) {
1673        LOG.trace("Yield {}", procedure, e);
1674        yieldProcedure(procedure);
1675        return;
1676      } catch (InterruptedException e) {
1677        LOG.trace("Yield interrupt {}", procedure, e);
1678        handleInterruptedException(procedure, e);
1679        yieldProcedure(procedure);
1680        return;
1681      } catch (Throwable e) {
1682        // Catch NullPointerExceptions or similar errors...
1683        String msg = "CODE-BUG: Uncaught runtime exception: " + procedure;
1684        LOG.error(msg, e);
1685        procedure.setFailure(new RemoteProcedureException(msg, e));
1686      }
1687
1688      if (!procedure.isFailed()) {
1689        if (subprocs != null) {
1690          if (subprocs.length == 1 && subprocs[0] == procedure) {
1691            // Procedure returned itself. Quick-shortcut for a state machine-like procedure;
1692            // i.e. we go around this loop again rather than go back out on the scheduler queue.
1693            subprocs = null;
1694            reExecute = true;
1695            LOG.trace("Short-circuit to next step on pid={}", procedure.getProcId());
1696          } else {
1697            // Yield the current procedure, and make the subprocedure runnable
1698            // subprocs may come back 'null'.
1699            subprocs = initializeChildren(procStack, procedure, subprocs);
1700            LOG.info("Initialized subprocedures=" +
1701              (subprocs == null? null:
1702                Stream.of(subprocs).map(e -> "{" + e.toString() + "}").
1703                collect(Collectors.toList()).toString()));
1704          }
1705        } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
1706          LOG.trace("Added to timeoutExecutor {}", procedure);
1707          timeoutExecutor.add(procedure);
1708        } else if (!suspended) {
1709          // No subtask, so we are done
1710          procedure.setState(ProcedureState.SUCCESS);
1711        }
1712      }
1713
1714      // Add the procedure to the stack
1715      procStack.addRollbackStep(procedure);
1716
1717      // allows to kill the executor before something is stored to the wal.
1718      // useful to test the procedure recovery.
1719      if (testing != null &&
1720        testing.shouldKillBeforeStoreUpdate(suspended, procedure.hasParent())) {
1721        kill("TESTING: Kill BEFORE store update: " + procedure);
1722      }
1723
1724      // TODO: The code here doesn't check if store is running before persisting to the store as
1725      // it relies on the method call below to throw RuntimeException to wind up the stack and
1726      // executor thread to stop. The statement following the method call below seems to check if
1727      // store is not running, to prevent scheduling children procedures, re-execution or yield
1728      // of this procedure. This may need more scrutiny and subsequent cleanup in future
1729      //
1730      // Commit the transaction even if a suspend (state may have changed). Note this append
1731      // can take a bunch of time to complete.
1732      if (procedure.needPersistence()) {
1733        updateStoreOnExec(procStack, procedure, subprocs);
1734      }
1735
1736      // if the store is not running we are aborting
1737      if (!store.isRunning()) {
1738        return;
1739      }
1740      // if the procedure is kind enough to pass the slot to someone else, yield
1741      if (procedure.isRunnable() && !suspended &&
1742          procedure.isYieldAfterExecutionStep(getEnvironment())) {
1743        yieldProcedure(procedure);
1744        return;
1745      }
1746
1747      assert (reExecute && subprocs == null) || !reExecute;
1748    } while (reExecute);
1749
1750    // Allows to kill the executor after something is stored to the WAL but before the below
1751    // state settings are done -- in particular the one on the end where we make parent
1752    // RUNNABLE again when its children are done; see countDownChildren.
1753    if (testing != null && testing.shouldKillAfterStoreUpdate(suspended)) {
1754      kill("TESTING: Kill AFTER store update: " + procedure);
1755    }
1756
1757    // Submit the new subprocedures
1758    if (subprocs != null && !procedure.isFailed()) {
1759      submitChildrenProcedures(subprocs);
1760    }
1761
1762    // we need to log the release lock operation before waking up the parent procedure, as there
1763    // could be race that the parent procedure may call updateStoreOnExec ahead of us and remove all
1764    // the sub procedures from store and cause problems...
1765    releaseLock(procedure, false);
1766
1767    // if the procedure is complete and has a parent, count down the children latch.
1768    // If 'suspended', do nothing to change state -- let other threads handle unsuspend event.
1769    if (!suspended && procedure.isFinished() && procedure.hasParent()) {
1770      countDownChildren(procStack, procedure);
1771    }
1772  }
1773
1774  private void kill(String msg) {
1775    LOG.debug(msg);
1776    stop();
1777    throw new RuntimeException(msg);
1778  }
1779
1780  private Procedure<TEnvironment>[] initializeChildren(RootProcedureState<TEnvironment> procStack,
1781      Procedure<TEnvironment> procedure, Procedure<TEnvironment>[] subprocs) {
1782    assert subprocs != null : "expected subprocedures";
1783    final long rootProcId = getRootProcedureId(procedure);
1784    for (int i = 0; i < subprocs.length; ++i) {
1785      Procedure<TEnvironment> subproc = subprocs[i];
1786      if (subproc == null) {
1787        String msg = "subproc[" + i + "] is null, aborting the procedure";
1788        procedure.setFailure(new RemoteProcedureException(msg,
1789          new IllegalArgumentIOException(msg)));
1790        return null;
1791      }
1792
1793      assert subproc.getState() == ProcedureState.INITIALIZING : subproc;
1794      subproc.setParentProcId(procedure.getProcId());
1795      subproc.setRootProcId(rootProcId);
1796      subproc.setProcId(nextProcId());
1797      procStack.addSubProcedure(subproc);
1798    }
1799
1800    if (!procedure.isFailed()) {
1801      procedure.setChildrenLatch(subprocs.length);
1802      switch (procedure.getState()) {
1803        case RUNNABLE:
1804          procedure.setState(ProcedureState.WAITING);
1805          break;
1806        case WAITING_TIMEOUT:
1807          timeoutExecutor.add(procedure);
1808          break;
1809        default:
1810          break;
1811      }
1812    }
1813    return subprocs;
1814  }
1815
1816  private void submitChildrenProcedures(Procedure<TEnvironment>[] subprocs) {
1817    for (int i = 0; i < subprocs.length; ++i) {
1818      Procedure<TEnvironment> subproc = subprocs[i];
1819      subproc.updateMetricsOnSubmit(getEnvironment());
1820      assert !procedures.containsKey(subproc.getProcId());
1821      procedures.put(subproc.getProcId(), subproc);
1822      scheduler.addFront(subproc);
1823    }
1824  }
1825
1826  private void countDownChildren(RootProcedureState<TEnvironment> procStack,
1827      Procedure<TEnvironment> procedure) {
1828    Procedure<TEnvironment> parent = procedures.get(procedure.getParentProcId());
1829    if (parent == null) {
1830      assert procStack.isRollingback();
1831      return;
1832    }
1833
1834    // If this procedure is the last child awake the parent procedure
1835    if (parent.tryRunnable()) {
1836      // If we succeeded in making the parent runnable -- i.e. all of its
1837      // children have completed, move parent to front of the queue.
1838      store.update(parent);
1839      scheduler.addFront(parent);
1840      LOG.info("Finished subprocedure pid={}, resume processing ppid={}",
1841        procedure.getProcId(), parent.getProcId());
1842      return;
1843    }
1844  }
1845
1846  private void updateStoreOnExec(RootProcedureState<TEnvironment> procStack,
1847      Procedure<TEnvironment> procedure, Procedure<TEnvironment>[] subprocs) {
1848    if (subprocs != null && !procedure.isFailed()) {
1849      if (LOG.isTraceEnabled()) {
1850        LOG.trace("Stored " + procedure + ", children " + Arrays.toString(subprocs));
1851      }
1852      store.insert(procedure, subprocs);
1853    } else {
1854      LOG.trace("Store update {}", procedure);
1855      if (procedure.isFinished() && !procedure.hasParent()) {
1856        // remove child procedures
1857        final long[] childProcIds = procStack.getSubprocedureIds();
1858        if (childProcIds != null) {
1859          store.delete(procedure, childProcIds);
1860          for (int i = 0; i < childProcIds.length; ++i) {
1861            procedures.remove(childProcIds[i]);
1862          }
1863        } else {
1864          store.update(procedure);
1865        }
1866      } else {
1867        store.update(procedure);
1868      }
1869    }
1870  }
1871
1872  private void handleInterruptedException(Procedure<TEnvironment> proc, InterruptedException e) {
1873    LOG.trace("Interrupt during {}. suspend and retry it later.", proc, e);
1874    // NOTE: We don't call Thread.currentThread().interrupt()
1875    // because otherwise all the subsequent calls e.g. Thread.sleep() will throw
1876    // the InterruptedException. If the master is going down, we will be notified
1877    // and the executor/store will be stopped.
1878    // (The interrupted procedure will be retried on the next run)
1879  }
1880
1881  private void execCompletionCleanup(Procedure<TEnvironment> proc) {
1882    final TEnvironment env = getEnvironment();
1883    if (proc.hasLock()) {
1884      LOG.warn("Usually this should not happen, we will release the lock before if the procedure" +
1885        " is finished, even if the holdLock is true, arrive here means we have some holes where" +
1886        " we do not release the lock. And the releaseLock below may fail since the procedure may" +
1887        " have already been deleted from the procedure store.");
1888      releaseLock(proc, true);
1889    }
1890    try {
1891      proc.completionCleanup(env);
1892    } catch (Throwable e) {
1893      // Catch NullPointerExceptions or similar errors...
1894      LOG.error("CODE-BUG: uncatched runtime exception for procedure: " + proc, e);
1895    }
1896  }
1897
1898  private void procedureFinished(Procedure<TEnvironment> proc) {
1899    // call the procedure completion cleanup handler
1900    execCompletionCleanup(proc);
1901
1902    CompletedProcedureRetainer<TEnvironment> retainer = new CompletedProcedureRetainer<>(proc);
1903
1904    // update the executor internal state maps
1905    if (!proc.shouldWaitClientAck(getEnvironment())) {
1906      retainer.setClientAckTime(0);
1907    }
1908
1909    completed.put(proc.getProcId(), retainer);
1910    rollbackStack.remove(proc.getProcId());
1911    procedures.remove(proc.getProcId());
1912
1913    // call the runnableSet completion cleanup handler
1914    try {
1915      scheduler.completionCleanup(proc);
1916    } catch (Throwable e) {
1917      // Catch NullPointerExceptions or similar errors...
1918      LOG.error("CODE-BUG: uncatched runtime exception for completion cleanup: {}", proc, e);
1919    }
1920
1921    // Notify the listeners
1922    sendProcedureFinishedNotification(proc.getProcId());
1923  }
1924
1925  RootProcedureState<TEnvironment> getProcStack(long rootProcId) {
1926    return rollbackStack.get(rootProcId);
1927  }
1928
1929  ProcedureScheduler getProcedureScheduler() {
1930    return scheduler;
1931  }
1932
1933  int getCompletedSize() {
1934    return completed.size();
1935  }
1936
1937  public IdLock getProcExecutionLock() {
1938    return procExecutionLock;
1939  }
1940
1941  // ==========================================================================
1942  //  Worker Thread
1943  // ==========================================================================
1944  private class WorkerThread extends StoppableThread {
1945    private final AtomicLong executionStartTime = new AtomicLong(Long.MAX_VALUE);
1946    private volatile Procedure<TEnvironment> activeProcedure;
1947
1948    public WorkerThread(ThreadGroup group) {
1949      this(group, "PEWorker-");
1950    }
1951
1952    protected WorkerThread(ThreadGroup group, String prefix) {
1953      super(group, prefix + workerId.incrementAndGet());
1954      setDaemon(true);
1955    }
1956
1957    @Override
1958    public void sendStopSignal() {
1959      scheduler.signalAll();
1960    }
1961    @Override
1962    public void run() {
1963      long lastUpdate = EnvironmentEdgeManager.currentTime();
1964      try {
1965        while (isRunning() && keepAlive(lastUpdate)) {
1966          @SuppressWarnings("unchecked")
1967          Procedure<TEnvironment> proc = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
1968          if (proc == null) {
1969            continue;
1970          }
1971          this.activeProcedure = proc;
1972          int activeCount = activeExecutorCount.incrementAndGet();
1973          int runningCount = store.setRunningProcedureCount(activeCount);
1974          LOG.trace("Execute pid={} runningCount={}, activeCount={}", proc.getProcId(),
1975            runningCount, activeCount);
1976          executionStartTime.set(EnvironmentEdgeManager.currentTime());
1977          IdLock.Entry lockEntry = procExecutionLock.getLockEntry(proc.getProcId());
1978          try {
1979            executeProcedure(proc);
1980          } catch (AssertionError e) {
1981            LOG.info("ASSERT pid=" + proc.getProcId(), e);
1982            throw e;
1983          } finally {
1984            procExecutionLock.releaseLockEntry(lockEntry);
1985            activeCount = activeExecutorCount.decrementAndGet();
1986            runningCount = store.setRunningProcedureCount(activeCount);
1987            LOG.trace("Halt pid={} runningCount={}, activeCount={}", proc.getProcId(),
1988              runningCount, activeCount);
1989            this.activeProcedure = null;
1990            lastUpdate = EnvironmentEdgeManager.currentTime();
1991            executionStartTime.set(Long.MAX_VALUE);
1992          }
1993        }
1994      } catch (Throwable t) {
1995        LOG.warn("Worker terminating UNNATURALLY {}", this.activeProcedure, t);
1996      } finally {
1997        LOG.trace("Worker terminated.");
1998      }
1999      workerThreads.remove(this);
2000    }
2001
2002    @Override
2003    public String toString() {
2004      Procedure<?> p = this.activeProcedure;
2005      return getName() + "(pid=" + (p == null? Procedure.NO_PROC_ID: p.getProcId() + ")");
2006    }
2007
2008    /**
2009     * @return the time since the current procedure is running
2010     */
2011    public long getCurrentRunTime() {
2012      return EnvironmentEdgeManager.currentTime() - executionStartTime.get();
2013    }
2014
2015    // core worker never timeout
2016    protected boolean keepAlive(long lastUpdate) {
2017      return true;
2018    }
2019  }
2020
2021  // A worker thread which can be added when core workers are stuck. Will timeout after
2022  // keepAliveTime if there is no procedure to run.
2023  private final class KeepAliveWorkerThread extends WorkerThread {
2024    public KeepAliveWorkerThread(ThreadGroup group) {
2025      super(group, "KeepAlivePEWorker-");
2026    }
2027
2028    @Override
2029    protected boolean keepAlive(long lastUpdate) {
2030      return EnvironmentEdgeManager.currentTime() - lastUpdate < keepAliveTime;
2031    }
2032  }
2033
2034  // ----------------------------------------------------------------------------
2035  // TODO-MAYBE: Should we provide a InlineChore to notify the store with the
2036  // full set of procedures pending and completed to write a compacted
2037  // version of the log (in case is a log)?
2038  // In theory no, procedures are have a short life, so at some point the store
2039  // will have the tracker saying everything is in the last log.
2040  // ----------------------------------------------------------------------------
2041
2042  private final class WorkerMonitor extends InlineChore {
2043    public static final String WORKER_MONITOR_INTERVAL_CONF_KEY =
2044        "hbase.procedure.worker.monitor.interval.msec";
2045    private static final int DEFAULT_WORKER_MONITOR_INTERVAL = 5000; // 5sec
2046
2047    public static final String WORKER_STUCK_THRESHOLD_CONF_KEY =
2048        "hbase.procedure.worker.stuck.threshold.msec";
2049    private static final int DEFAULT_WORKER_STUCK_THRESHOLD = 10000; // 10sec
2050
2051    public static final String WORKER_ADD_STUCK_PERCENTAGE_CONF_KEY =
2052        "hbase.procedure.worker.add.stuck.percentage";
2053    private static final float DEFAULT_WORKER_ADD_STUCK_PERCENTAGE = 0.5f; // 50% stuck
2054
2055    private float addWorkerStuckPercentage = DEFAULT_WORKER_ADD_STUCK_PERCENTAGE;
2056    private int timeoutInterval = DEFAULT_WORKER_MONITOR_INTERVAL;
2057    private int stuckThreshold = DEFAULT_WORKER_STUCK_THRESHOLD;
2058
2059    public WorkerMonitor() {
2060      refreshConfig();
2061    }
2062
2063    @Override
2064    public void run() {
2065      final int stuckCount = checkForStuckWorkers();
2066      checkThreadCount(stuckCount);
2067
2068      // refresh interval (poor man dynamic conf update)
2069      refreshConfig();
2070    }
2071
2072    private int checkForStuckWorkers() {
2073      // check if any of the worker is stuck
2074      int stuckCount = 0;
2075      for (WorkerThread worker : workerThreads) {
2076        if (worker.getCurrentRunTime() < stuckThreshold) {
2077          continue;
2078        }
2079
2080        // WARN the worker is stuck
2081        stuckCount++;
2082        LOG.warn("Worker stuck {}, run time {}", worker,
2083          StringUtils.humanTimeDiff(worker.getCurrentRunTime()));
2084      }
2085      return stuckCount;
2086    }
2087
2088    private void checkThreadCount(final int stuckCount) {
2089      // nothing to do if there are no runnable tasks
2090      if (stuckCount < 1 || !scheduler.hasRunnables()) {
2091        return;
2092      }
2093
2094      // add a new thread if the worker stuck percentage exceed the threshold limit
2095      // and every handler is active.
2096      final float stuckPerc = ((float) stuckCount) / workerThreads.size();
2097      // let's add new worker thread more aggressively, as they will timeout finally if there is no
2098      // work to do.
2099      if (stuckPerc >= addWorkerStuckPercentage && workerThreads.size() < maxPoolSize) {
2100        final KeepAliveWorkerThread worker = new KeepAliveWorkerThread(threadGroup);
2101        workerThreads.add(worker);
2102        worker.start();
2103        LOG.debug("Added new worker thread {}", worker);
2104      }
2105    }
2106
2107    private void refreshConfig() {
2108      addWorkerStuckPercentage = conf.getFloat(WORKER_ADD_STUCK_PERCENTAGE_CONF_KEY,
2109          DEFAULT_WORKER_ADD_STUCK_PERCENTAGE);
2110      timeoutInterval = conf.getInt(WORKER_MONITOR_INTERVAL_CONF_KEY,
2111        DEFAULT_WORKER_MONITOR_INTERVAL);
2112      stuckThreshold = conf.getInt(WORKER_STUCK_THRESHOLD_CONF_KEY,
2113        DEFAULT_WORKER_STUCK_THRESHOLD);
2114    }
2115
2116    @Override
2117    public int getTimeoutInterval() {
2118      return timeoutInterval;
2119    }
2120  }
2121}