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