View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.procedure2;
20  
21  import java.io.IOException;
22  import java.io.InputStream;
23  import java.io.OutputStream;
24  import java.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.Collections;
27  import java.util.Iterator;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.HashSet;
31  import java.util.TreeSet;
32  import java.util.concurrent.atomic.AtomicBoolean;
33  import java.util.concurrent.atomic.AtomicInteger;
34  import java.util.concurrent.atomic.AtomicLong;
35  import java.util.concurrent.locks.ReentrantLock;
36  import java.util.concurrent.ConcurrentHashMap;
37  import java.util.concurrent.CopyOnWriteArrayList;
38  import java.util.concurrent.TimeUnit;
39  
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.hbase.ProcedureInfo;
44  import org.apache.hadoop.hbase.classification.InterfaceAudience;
45  import org.apache.hadoop.hbase.classification.InterfaceStability;
46  import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
47  import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
48  import org.apache.hadoop.hbase.procedure2.util.StringUtils;
49  import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue;
50  import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue.TimeoutRetriever;
51  import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
52  import org.apache.hadoop.hbase.security.User;
53  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
54  import org.apache.hadoop.hbase.util.Pair;
55  
56  import com.google.common.base.Preconditions;
57  
58  /**
59   * Thread Pool that executes the submitted procedures.
60   * The executor has a ProcedureStore associated.
61   * Each operation is logged and on restart the pending procedures are resumed.
62   *
63   * Unless the Procedure code throws an error (e.g. invalid user input)
64   * the procedure will complete (at some point in time), On restart the pending
65   * procedures are resumed and the once failed will be rolledback.
66   *
67   * The user can add procedures to the executor via submitProcedure(proc)
68   * check for the finished state via isFinished(procId)
69   * and get the result via getResult(procId)
70   */
71  @InterfaceAudience.Private
72  @InterfaceStability.Evolving
73  public class ProcedureExecutor<TEnvironment> {
74    private static final Log LOG = LogFactory.getLog(ProcedureExecutor.class);
75  
76    Testing testing = null;
77    public static class Testing {
78      protected boolean killBeforeStoreUpdate = false;
79      protected boolean toggleKillBeforeStoreUpdate = false;
80  
81      protected boolean shouldKillBeforeStoreUpdate() {
82        final boolean kill = this.killBeforeStoreUpdate;
83        if (this.toggleKillBeforeStoreUpdate) {
84          this.killBeforeStoreUpdate = !kill;
85          LOG.warn("Toggle Kill before store update to: " + this.killBeforeStoreUpdate);
86        }
87        return kill;
88      }
89    }
90  
91    public interface ProcedureExecutorListener {
92      void procedureLoaded(long procId);
93      void procedureAdded(long procId);
94      void procedureFinished(long procId);
95    }
96  
97    /**
98     * Used by the TimeoutBlockingQueue to get the timeout interval of the procedure
99     */
100   private static class ProcedureTimeoutRetriever implements TimeoutRetriever<Procedure> {
101     @Override
102     public long getTimeout(Procedure proc) {
103       return proc.getTimeRemaining();
104     }
105 
106     @Override
107     public TimeUnit getTimeUnit(Procedure proc) {
108       return TimeUnit.MILLISECONDS;
109     }
110   }
111 
112   /**
113    * Internal cleaner that removes the completed procedure results after a TTL.
114    * NOTE: This is a special case handled in timeoutLoop().
115    *
116    * Since the client code looks more or less like:
117    *   procId = master.doOperation()
118    *   while (master.getProcResult(procId) == ProcInProgress);
119    * The master should not throw away the proc result as soon as the procedure is done
120    * but should wait a result request from the client (see executor.removeResult(procId))
121    * The client will call something like master.isProcDone() or master.getProcResult()
122    * which will return the result/state to the client, and it will mark the completed
123    * proc as ready to delete. note that the client may not receive the response from
124    * the master (e.g. master failover) so, if we delay a bit the real deletion of
125    * the proc result the client will be able to get the result the next try.
126    */
127   private static class CompletedProcedureCleaner<TEnvironment> extends Procedure<TEnvironment> {
128     private static final Log LOG = LogFactory.getLog(CompletedProcedureCleaner.class);
129 
130     private static final String CLEANER_INTERVAL_CONF_KEY = "hbase.procedure.cleaner.interval";
131     private static final int DEFAULT_CLEANER_INTERVAL = 30 * 1000; // 30sec
132 
133     private static final String EVICT_TTL_CONF_KEY = "hbase.procedure.cleaner.evict.ttl";
134     private static final int DEFAULT_EVICT_TTL = 15 * 60000; // 15min
135 
136     private static final String EVICT_ACKED_TTL_CONF_KEY ="hbase.procedure.cleaner.acked.evict.ttl";
137     private static final int DEFAULT_ACKED_EVICT_TTL = 5 * 60000; // 5min
138 
139     private final Map<Long, ProcedureInfo> completed;
140     private final ProcedureStore store;
141     private final Configuration conf;
142 
143     public CompletedProcedureCleaner(final Configuration conf, final ProcedureStore store,
144         final Map<Long, ProcedureInfo> completedMap) {
145       // set the timeout interval that triggers the periodic-procedure
146       setTimeout(conf.getInt(CLEANER_INTERVAL_CONF_KEY, DEFAULT_CLEANER_INTERVAL));
147       this.completed = completedMap;
148       this.store = store;
149       this.conf = conf;
150     }
151 
152     public void periodicExecute(final TEnvironment env) {
153       if (completed.isEmpty()) {
154         if (LOG.isDebugEnabled()) {
155           LOG.debug("No completed procedures to cleanup.");
156         }
157         return;
158       }
159 
160       final long evictTtl = conf.getInt(EVICT_TTL_CONF_KEY, DEFAULT_EVICT_TTL);
161       final long evictAckTtl = conf.getInt(EVICT_ACKED_TTL_CONF_KEY, DEFAULT_ACKED_EVICT_TTL);
162 
163       long now = EnvironmentEdgeManager.currentTime();
164       Iterator<Map.Entry<Long, ProcedureInfo>> it = completed.entrySet().iterator();
165       while (it.hasNext() && store.isRunning()) {
166         Map.Entry<Long, ProcedureInfo> entry = it.next();
167         ProcedureInfo result = entry.getValue();
168 
169         // TODO: Select TTL based on Procedure type
170         if ((result.hasClientAckTime() && (now - result.getClientAckTime()) >= evictAckTtl) ||
171             (now - result.getLastUpdate()) >= evictTtl) {
172           if (LOG.isDebugEnabled()) {
173             LOG.debug("Evict completed procedure " + entry.getKey());
174           }
175           store.delete(entry.getKey());
176           it.remove();
177         }
178       }
179     }
180 
181     @Override
182     protected Procedure[] execute(final TEnvironment env) {
183       throw new UnsupportedOperationException();
184     }
185 
186     @Override
187     protected void rollback(final TEnvironment env) {
188       throw new UnsupportedOperationException();
189     }
190 
191     @Override
192     protected boolean abort(final TEnvironment env) {
193       throw new UnsupportedOperationException();
194     }
195 
196     @Override
197     public void serializeStateData(final OutputStream stream) {
198       throw new UnsupportedOperationException();
199     }
200 
201     @Override
202     public void deserializeStateData(final InputStream stream) {
203       throw new UnsupportedOperationException();
204     }
205   }
206 
207   /**
208    * Map the the procId returned by submitProcedure(), the Root-ProcID, to the ProcedureInfo.
209    * Once a Root-Procedure completes (success or failure), the result will be added to this map.
210    * The user of ProcedureExecutor should call getResult(procId) to get the result.
211    */
212   private final ConcurrentHashMap<Long, ProcedureInfo> completed =
213     new ConcurrentHashMap<Long, ProcedureInfo>();
214 
215   /**
216    * Map the the procId returned by submitProcedure(), the Root-ProcID, to the RootProcedureState.
217    * The RootProcedureState contains the execution stack of the Root-Procedure,
218    * It is added to the map by submitProcedure() and removed on procedure completion.
219    */
220   private final ConcurrentHashMap<Long, RootProcedureState> rollbackStack =
221     new ConcurrentHashMap<Long, RootProcedureState>();
222 
223   /**
224    * Helper map to lookup the live procedures by ID.
225    * This map contains every procedure. root-procedures and subprocedures.
226    */
227   private final ConcurrentHashMap<Long, Procedure> procedures =
228     new ConcurrentHashMap<Long, Procedure>();
229 
230   /**
231    * Timeout Queue that contains Procedures in a WAITING_TIMEOUT state
232    * or periodic procedures.
233    */
234   private final TimeoutBlockingQueue<Procedure> waitingTimeout =
235     new TimeoutBlockingQueue<Procedure>(new ProcedureTimeoutRetriever());
236 
237   /**
238    * Queue that contains runnable procedures.
239    */
240   private final ProcedureRunnableSet runnables;
241 
242   // TODO
243   private final ReentrantLock submitLock = new ReentrantLock();
244   private final AtomicLong lastProcId = new AtomicLong(-1);
245 
246   private final CopyOnWriteArrayList<ProcedureExecutorListener> listeners =
247     new CopyOnWriteArrayList<ProcedureExecutorListener>();
248 
249   private final AtomicInteger activeExecutorCount = new AtomicInteger(0);
250   private final AtomicBoolean running = new AtomicBoolean(false);
251   private final TEnvironment environment;
252   private final ProcedureStore store;
253   private final Configuration conf;
254 
255   private Thread[] threads;
256 
257   public ProcedureExecutor(final Configuration conf, final TEnvironment environment,
258       final ProcedureStore store) {
259     this(conf, environment, store, new ProcedureSimpleRunQueue());
260   }
261 
262   public ProcedureExecutor(final Configuration conf, final TEnvironment environment,
263       final ProcedureStore store, final ProcedureRunnableSet runqueue) {
264     this.environment = environment;
265     this.runnables = runqueue;
266     this.store = store;
267     this.conf = conf;
268   }
269 
270   private List<Map.Entry<Long, RootProcedureState>> load() throws IOException {
271     Preconditions.checkArgument(completed.isEmpty());
272     Preconditions.checkArgument(rollbackStack.isEmpty());
273     Preconditions.checkArgument(procedures.isEmpty());
274     Preconditions.checkArgument(waitingTimeout.isEmpty());
275     Preconditions.checkArgument(runnables.size() == 0);
276 
277     // 1. Load the procedures
278     Iterator<Procedure> loader = store.load();
279     if (loader == null) {
280       lastProcId.set(0);
281       return null;
282     }
283 
284     long logMaxProcId = 0;
285     int runnablesCount = 0;
286     while (loader.hasNext()) {
287       Procedure proc = loader.next();
288       proc.beforeReplay(getEnvironment());
289       procedures.put(proc.getProcId(), proc);
290       logMaxProcId = Math.max(logMaxProcId, proc.getProcId());
291       if (LOG.isDebugEnabled()) {
292         LOG.debug("Loading procedure state=" + proc.getState() +
293             " isFailed=" + proc.hasException() + ": " + proc);
294       }
295       if (!proc.hasParent() && !proc.isFinished()) {
296         rollbackStack.put(proc.getProcId(), new RootProcedureState());
297       }
298       if (proc.getState() == ProcedureState.RUNNABLE) {
299         runnablesCount++;
300       }
301     }
302     assert lastProcId.get() < 0;
303     lastProcId.set(logMaxProcId);
304 
305     // 2. Initialize the stacks
306     TreeSet<Procedure> runnableSet = null;
307     HashSet<Procedure> waitingSet = null;
308     for (final Procedure proc: procedures.values()) {
309       Long rootProcId = getRootProcedureId(proc);
310       if (rootProcId == null) {
311         // The 'proc' was ready to run but the root procedure was rolledback?
312         runnables.addBack(proc);
313         continue;
314       }
315 
316       if (!proc.hasParent() && proc.isFinished()) {
317         if (LOG.isDebugEnabled()) {
318           LOG.debug("The procedure is completed state=" + proc.getState() +
319               " isFailed=" + proc.hasException() + ": " + proc);
320         }
321         assert !rollbackStack.containsKey(proc.getProcId());
322         completed.put(proc.getProcId(), Procedure.createProcedureInfo(proc));
323         continue;
324       }
325 
326       if (proc.hasParent() && !proc.isFinished()) {
327         Procedure parent = procedures.get(proc.getParentProcId());
328         // corrupted procedures are handled later at step 3
329         if (parent != null) {
330           parent.incChildrenLatch();
331         }
332       }
333 
334       RootProcedureState procStack = rollbackStack.get(rootProcId);
335       procStack.loadStack(proc);
336 
337       switch (proc.getState()) {
338         case RUNNABLE:
339           if (runnableSet == null) {
340             runnableSet = new TreeSet<Procedure>();
341           }
342           runnableSet.add(proc);
343           break;
344         case WAITING_TIMEOUT:
345           if (waitingSet == null) {
346             waitingSet = new HashSet<Procedure>();
347           }
348           waitingSet.add(proc);
349           break;
350         case FINISHED:
351           if (proc.hasException()) {
352             // add the proc to the runnables to perform the rollback
353             runnables.addBack(proc);
354             break;
355           }
356         case ROLLEDBACK:
357         case INITIALIZING:
358           String msg = "Unexpected " + proc.getState() + " state for " + proc;
359           LOG.error(msg);
360           throw new UnsupportedOperationException(msg);
361         default:
362           break;
363       }
364     }
365 
366     // 3. Validate the stacks
367     List<Map.Entry<Long, RootProcedureState>> corrupted = null;
368     Iterator<Map.Entry<Long, RootProcedureState>> itStack = rollbackStack.entrySet().iterator();
369     while (itStack.hasNext()) {
370       Map.Entry<Long, RootProcedureState> entry = itStack.next();
371       RootProcedureState procStack = entry.getValue();
372       if (procStack.isValid()) continue;
373 
374       for (Procedure proc: procStack.getSubprocedures()) {
375         procedures.remove(proc.getProcId());
376         if (runnableSet != null) runnableSet.remove(proc);
377         if (waitingSet != null) waitingSet.remove(proc);
378       }
379       itStack.remove();
380       if (corrupted == null) {
381         corrupted = new ArrayList<Map.Entry<Long, RootProcedureState>>();
382       }
383       corrupted.add(entry);
384     }
385 
386     // 4. Push the runnables
387     if (runnableSet != null) {
388       // TODO: See ProcedureWALFormatReader.readInitEntry() some procedure
389       // may be started way before this stuff.
390       for (Procedure proc: runnableSet) {
391         if (!proc.hasParent()) {
392           sendProcedureLoadedNotification(proc.getProcId());
393         }
394         runnables.addBack(proc);
395       }
396     }
397     return corrupted;
398   }
399 
400   public void start(int numThreads) throws IOException {
401     if (running.getAndSet(true)) {
402       LOG.warn("Already running");
403       return;
404     }
405 
406     // We have numThreads executor + one timer thread used for timing out
407     // procedures and triggering periodic procedures.
408     threads = new Thread[numThreads + 1];
409     LOG.info("Starting procedure executor threads=" + threads.length);
410 
411     // Initialize procedures executor
412     for (int i = 0; i < numThreads; ++i) {
413       threads[i] = new Thread("ProcedureExecutorThread-" + i) {
414         @Override
415         public void run() {
416           execLoop();
417         }
418       };
419     }
420 
421     // Initialize procedures timeout handler (this is the +1 thread)
422     threads[numThreads] = new Thread("ProcedureExecutorTimeout") {
423       @Override
424       public void run() {
425         timeoutLoop();
426       }
427     };
428 
429     // Acquire the store lease.
430     store.recoverLease();
431 
432     // TODO: Split in two steps.
433     // TODO: Handle corrupted procedure returned (probably just a WARN)
434     // The first one will make sure that we have the latest id,
435     // so we can start the threads and accept new procedures.
436     // The second step will do the actual load of old procedures.
437     load();
438 
439     // Start the executors. Here we must have the lastProcId set.
440     for (int i = 0; i < threads.length; ++i) {
441       threads[i].start();
442     }
443 
444     // Add completed cleaner
445     waitingTimeout.add(new CompletedProcedureCleaner(conf, store, completed));
446   }
447 
448   public void stop() {
449     if (!running.getAndSet(false)) {
450       return;
451     }
452 
453     LOG.info("Stopping the procedure executor");
454     runnables.signalAll();
455     waitingTimeout.signalAll();
456   }
457 
458   public void join() {
459     boolean interrupted = false;
460 
461     for (int i = 0; i < threads.length; ++i) {
462       try {
463         threads[i].join();
464       } catch (InterruptedException ex) {
465         interrupted = true;
466       }
467     }
468 
469     if (interrupted) {
470       Thread.currentThread().interrupt();
471     }
472 
473     completed.clear();
474     rollbackStack.clear();
475     procedures.clear();
476     waitingTimeout.clear();
477     runnables.clear();
478     lastProcId.set(-1);
479   }
480 
481   public boolean isRunning() {
482     return running.get();
483   }
484 
485   /**
486    * @return the number of execution threads.
487    */
488   public int getNumThreads() {
489     return threads == null ? 0 : (threads.length - 1);
490   }
491 
492   public int getActiveExecutorCount() {
493     return activeExecutorCount.get();
494   }
495 
496   public TEnvironment getEnvironment() {
497     return this.environment;
498   }
499 
500   public ProcedureStore getStore() {
501     return this.store;
502   }
503 
504   public void registerListener(ProcedureExecutorListener listener) {
505     this.listeners.add(listener);
506   }
507 
508   public boolean unregisterListener(ProcedureExecutorListener listener) {
509     return this.listeners.remove(listener);
510   }
511 
512   /**
513    * List procedures.
514    * @return the procedures in a list
515    */
516   public List<ProcedureInfo> listProcedures() {
517     List<ProcedureInfo> procedureLists =
518         new ArrayList<ProcedureInfo>(procedures.size() + completed.size());
519     for (java.util.Map.Entry<Long, Procedure> p: procedures.entrySet()) {
520       procedureLists.add(Procedure.createProcedureInfo(p.getValue()));
521     }
522     for (java.util.Map.Entry<Long, ProcedureInfo> e: completed.entrySet()) {
523       // Note: The procedure could show up twice in the list with different state, as
524       // it could complete after we walk through procedures list and insert into
525       // procedureList - it is ok, as we will use the information in the ProcedureInfo
526       // to figure it out; to prevent this would increase the complexity of the logic.
527       procedureLists.add(e.getValue());
528     }
529     return procedureLists;
530   }
531 
532   /**
533    * Add a new root-procedure to the executor.
534    * @param proc the new procedure to execute.
535    * @return the procedure id, that can be used to monitor the operation
536    */
537   public long submitProcedure(final Procedure proc) {
538     Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING);
539     Preconditions.checkArgument(isRunning());
540     Preconditions.checkArgument(lastProcId.get() >= 0);
541     Preconditions.checkArgument(!proc.hasParent());
542 
543     // Initialize the Procedure ID
544     proc.setProcId(nextProcId());
545 
546     // Commit the transaction
547     store.insert(proc, null);
548     if (LOG.isDebugEnabled()) {
549       LOG.debug("Procedure " + proc + " added to the store.");
550     }
551 
552     // Create the rollback stack for the procedure
553     RootProcedureState stack = new RootProcedureState();
554     rollbackStack.put(proc.getProcId(), stack);
555 
556     // Submit the new subprocedures
557     assert !procedures.containsKey(proc.getProcId());
558     procedures.put(proc.getProcId(), proc);
559     sendProcedureAddedNotification(proc.getProcId());
560     runnables.addBack(proc);
561     return proc.getProcId();
562   }
563 
564   public ProcedureInfo getResult(final long procId) {
565     return completed.get(procId);
566   }
567 
568   /**
569    * Return true if the procedure is finished.
570    * The state may be "completed successfully" or "failed and rolledback".
571    * Use getResult() to check the state or get the result data.
572    * @param procId the ID of the procedure to check
573    * @return true if the procedure execution is finished, otherwise false.
574    */
575   public boolean isFinished(final long procId) {
576     return completed.containsKey(procId);
577   }
578 
579   /**
580    * Return true if the procedure is started.
581    * @param procId the ID of the procedure to check
582    * @return true if the procedure execution is started, otherwise false.
583    */
584   public boolean isStarted(final long procId) {
585     Procedure proc = procedures.get(procId);
586     if (proc == null) {
587       return completed.get(procId) != null;
588     }
589     return proc.wasExecuted();
590   }
591 
592   /**
593    * Mark the specified completed procedure, as ready to remove.
594    * @param procId the ID of the procedure to remove
595    */
596   public void removeResult(final long procId) {
597     ProcedureInfo result = completed.get(procId);
598     if (result == null) {
599       assert !procedures.containsKey(procId) : "procId=" + procId + " is still running";
600       if (LOG.isDebugEnabled()) {
601         LOG.debug("Procedure procId=" + procId + " already removed by the cleaner.");
602       }
603       return;
604     }
605 
606     // The CompletedProcedureCleaner will take care of deletion, once the TTL is expired.
607     result.setClientAckTime(EnvironmentEdgeManager.currentTime());
608   }
609 
610   /**
611    * Send an abort notification the specified procedure.
612    * Depending on the procedure implementation the abort can be considered or ignored.
613    * @param procId the procedure to abort
614    * @return true if the procedure exist and has received the abort, otherwise false.
615    */
616   public boolean abort(final long procId) {
617     return abort(procId, true);
618   }
619 
620   /**
621    * Send an abort notification the specified procedure.
622    * Depending on the procedure implementation the abort can be considered or ignored.
623    * @param procId the procedure to abort
624    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
625    * @return true if the procedure exist and has received the abort, otherwise false.
626    */
627   public boolean abort(final long procId, final boolean mayInterruptIfRunning) {
628     Procedure proc = procedures.get(procId);
629     if (proc != null) {
630       if (!mayInterruptIfRunning && proc.wasExecuted()) {
631         return false;
632       } else {
633         return proc.abort(getEnvironment());
634       }
635     }
636     return false;
637   }
638 
639   /**
640    * Check if the user is this procedure's owner
641    * @param procId the target procedure
642    * @param user the user
643    * @return true if the user is the owner of the procedure,
644    *   false otherwise or the owner is unknown.
645    */
646   public boolean isProcedureOwner(final long procId, final User user) {
647     if (user == null) {
648       return false;
649     }
650 
651     Procedure proc = procedures.get(procId);
652     if (proc != null) {
653       return proc.getOwner().equals(user.getShortName());
654     }
655     ProcedureInfo procInfo = completed.get(procId);
656     if (procInfo == null) {
657       // Procedure either does not exist or has already completed and got cleaned up.
658       // At this time, we cannot check the owner of the procedure
659       return false;
660     }
661     return ProcedureInfo.isProcedureOwner(procInfo, user);
662   }
663 
664   public Map<Long, ProcedureInfo> getResults() {
665     return Collections.unmodifiableMap(completed);
666   }
667 
668   public Procedure getProcedure(final long procId) {
669     return procedures.get(procId);
670   }
671 
672   protected ProcedureRunnableSet getRunnableSet() {
673     return runnables;
674   }
675 
676   /**
677    * Execution loop (N threads)
678    * while the executor is in a running state,
679    * fetch a procedure from the runnables queue and start the execution.
680    */
681   private void execLoop() {
682     while (isRunning()) {
683       Long procId = runnables.poll();
684       Procedure proc = procId != null ? procedures.get(procId) : null;
685       if (proc == null) continue;
686 
687       try {
688         activeExecutorCount.incrementAndGet();
689         execLoop(proc);
690       } finally {
691         activeExecutorCount.decrementAndGet();
692       }
693     }
694   }
695 
696   private void execLoop(Procedure proc) {
697     if (LOG.isTraceEnabled()) {
698       LOG.trace("Trying to start the execution of " + proc);
699     }
700 
701     Long rootProcId = getRootProcedureId(proc);
702     if (rootProcId == null) {
703       // The 'proc' was ready to run but the root procedure was rolledback
704       executeRollback(proc);
705       return;
706     }
707 
708     RootProcedureState procStack = rollbackStack.get(rootProcId);
709     if (procStack == null) return;
710 
711     do {
712       // Try to acquire the execution
713       if (!procStack.acquire(proc)) {
714         if (procStack.setRollback()) {
715           // we have the 'rollback-lock' we can start rollingback
716           if (!executeRollback(rootProcId, procStack)) {
717             procStack.unsetRollback();
718             runnables.yield(proc);
719           }
720         } else {
721           // if we can't rollback means that some child is still running.
722           // the rollback will be executed after all the children are done.
723           // If the procedure was never executed, remove and mark it as rolledback.
724           if (!proc.wasExecuted()) {
725             if (!executeRollback(proc)) {
726               runnables.yield(proc);
727             }
728           }
729         }
730         break;
731       }
732 
733       // Execute the procedure
734       assert proc.getState() == ProcedureState.RUNNABLE;
735       if (proc.acquireLock(getEnvironment())) {
736         execProcedure(procStack, proc);
737         proc.releaseLock(getEnvironment());
738       } else {
739         runnables.yield(proc);
740       }
741       procStack.release(proc);
742 
743       // allows to kill the executor before something is stored to the wal.
744       // useful to test the procedure recovery.
745       if (testing != null && !isRunning()) {
746         break;
747       }
748 
749       if (proc.isSuccess()) {
750         if (LOG.isDebugEnabled()) {
751           LOG.debug("Procedure completed in " +
752               StringUtils.humanTimeDiff(proc.elapsedTime()) + ": " + proc);
753         }
754         // Finalize the procedure state
755         if (proc.getProcId() == rootProcId) {
756           procedureFinished(proc);
757         }
758         break;
759       }
760     } while (procStack.isFailed());
761   }
762 
763   private void timeoutLoop() {
764     while (isRunning()) {
765       Procedure proc = waitingTimeout.poll();
766       if (proc == null) continue;
767 
768       if (proc.getTimeRemaining() > 100) {
769         // got an early wake, maybe a stop?
770         // re-enqueue the task in case was not a stop or just a signal
771         waitingTimeout.add(proc);
772         continue;
773       }
774 
775       // ----------------------------------------------------------------------------
776       // TODO-MAYBE: Should we provide a notification to the store with the
777       // full set of procedures pending and completed to write a compacted
778       // version of the log (in case is a log)?
779       // In theory no, procedures are have a short life, so at some point the store
780       // will have the tracker saying everything is in the last log.
781       // ----------------------------------------------------------------------------
782 
783       // The CompletedProcedureCleaner is a special case, and it acts as a chore.
784       // instead of bringing the Chore class in, we reuse this timeout thread for
785       // this special case.
786       if (proc instanceof CompletedProcedureCleaner) {
787         try {
788           ((CompletedProcedureCleaner)proc).periodicExecute(getEnvironment());
789         } catch (Throwable e) {
790           LOG.error("Ignoring CompletedProcedureCleaner exception: " + e.getMessage(), e);
791         }
792         proc.setStartTime(EnvironmentEdgeManager.currentTime());
793         waitingTimeout.add(proc);
794         continue;
795       }
796 
797       // The procedure received an "abort-timeout", call abort() and
798       // add the procedure back in the queue for rollback.
799       if (proc.setTimeoutFailure()) {
800         long rootProcId = Procedure.getRootProcedureId(procedures, proc);
801         RootProcedureState procStack = rollbackStack.get(rootProcId);
802         procStack.abort();
803         store.update(proc);
804         runnables.addFront(proc);
805         continue;
806       }
807     }
808   }
809 
810   /**
811    * Execute the rollback of the full procedure stack.
812    * Once the procedure is rolledback, the root-procedure will be visible as
813    * finished to user, and the result will be the fatal exception.
814    */
815   private boolean executeRollback(final long rootProcId, final RootProcedureState procStack) {
816     Procedure rootProc = procedures.get(rootProcId);
817     RemoteProcedureException exception = rootProc.getException();
818     if (exception == null) {
819       exception = procStack.getException();
820       rootProc.setFailure(exception);
821       store.update(rootProc);
822     }
823 
824     List<Procedure> subprocStack = procStack.getSubprocedures();
825     assert subprocStack != null : "Called rollback with no steps executed rootProc=" + rootProc;
826 
827     int stackTail = subprocStack.size();
828     boolean reuseLock = false;
829     while (stackTail --> 0) {
830       final Procedure proc = subprocStack.get(stackTail);
831 
832       if (!reuseLock && !proc.acquireLock(getEnvironment())) {
833         // can't take a lock on the procedure, add the root-proc back on the
834         // queue waiting for the lock availability
835         return false;
836       }
837 
838       boolean abortRollback = !executeRollback(proc);
839       abortRollback |= !isRunning() || !store.isRunning();
840 
841       // If the next procedure is the same to this one
842       // (e.g. StateMachineProcedure reuse the same instance)
843       // we can avoid to lock/unlock each step
844       reuseLock = stackTail > 0 && (subprocStack.get(stackTail - 1) == proc) && !abortRollback;
845       if (!reuseLock) {
846         proc.releaseLock(getEnvironment());
847       }
848 
849       // allows to kill the executor before something is stored to the wal.
850       // useful to test the procedure recovery.
851       if (abortRollback) {
852         return false;
853       }
854 
855       subprocStack.remove(stackTail);
856     }
857 
858     // Finalize the procedure state
859     LOG.info("Rolledback procedure " + rootProc +
860              " exec-time=" + StringUtils.humanTimeDiff(rootProc.elapsedTime()) +
861              " exception=" + exception.getMessage());
862     procedureFinished(rootProc);
863     return true;
864   }
865 
866   /**
867    * Execute the rollback of the procedure step.
868    * It updates the store with the new state (stack index)
869    * or will remove completly the procedure in case it is a child.
870    */
871   private boolean executeRollback(final Procedure proc) {
872     try {
873       proc.doRollback(getEnvironment());
874     } catch (IOException e) {
875       if (LOG.isDebugEnabled()) {
876         LOG.debug("rollback attempt failed for " + proc, e);
877       }
878       return false;
879     } catch (Throwable e) {
880       // Catch NullPointerExceptions or similar errors...
881       LOG.fatal("CODE-BUG: Uncatched runtime exception for procedure: " + proc, e);
882     }
883 
884     // allows to kill the executor before something is stored to the wal.
885     // useful to test the procedure recovery.
886     if (testing != null && testing.shouldKillBeforeStoreUpdate()) {
887       if (LOG.isDebugEnabled()) {
888         LOG.debug("TESTING: Kill before store update");
889       }
890       stop();
891       return false;
892     }
893 
894     if (proc.removeStackIndex()) {
895       proc.setState(ProcedureState.ROLLEDBACK);
896       if (proc.hasParent()) {
897         store.delete(proc.getProcId());
898         procedures.remove(proc.getProcId());
899       } else {
900         store.update(proc);
901       }
902     } else {
903       store.update(proc);
904     }
905     return true;
906   }
907 
908   /**
909    * Executes the specified procedure
910    *  - calls the doExecute() of the procedure
911    *  - if the procedure execution didn't fail (e.g. invalid user input)
912    *     - ...and returned subprocedures
913    *        - the subprocedures are initialized.
914    *        - the subprocedures are added to the store
915    *        - the subprocedures are added to the runnable queue
916    *        - the procedure is now in a WAITING state, waiting for the subprocedures to complete
917    *     - ...if there are no subprocedure
918    *        - the procedure completed successfully
919    *        - if there is a parent (WAITING)
920    *            - the parent state will be set to RUNNABLE
921    *  - in case of failure
922    *    - the store is updated with the new state
923    *    - the executor (caller of this method) will start the rollback of the procedure
924    */
925   private void execProcedure(final RootProcedureState procStack, final Procedure procedure) {
926     Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE);
927 
928     // Execute the procedure
929     boolean reExecute = false;
930     Procedure[] subprocs = null;
931     do {
932       reExecute = false;
933       try {
934         subprocs = procedure.doExecute(getEnvironment());
935         if (subprocs != null && subprocs.length == 0) {
936           subprocs = null;
937         }
938       } catch (ProcedureYieldException e) {
939         if (LOG.isTraceEnabled()) {
940           LOG.trace("Yield procedure: " + procedure);
941         }
942         runnables.yield(procedure);
943         return;
944       } catch (Throwable e) {
945         // Catch NullPointerExceptions or similar errors...
946         String msg = "CODE-BUG: Uncatched runtime exception for procedure: " + procedure;
947         LOG.error(msg, e);
948         procedure.setFailure(new RemoteProcedureException(msg, e));
949       }
950 
951       if (!procedure.isFailed()) {
952         if (subprocs != null) {
953           if (subprocs.length == 1 && subprocs[0] == procedure) {
954             // quick-shortcut for a state machine like procedure
955             subprocs = null;
956             reExecute = true;
957           } else {
958             // yield the current procedure, and make the subprocedure runnable
959             for (int i = 0; i < subprocs.length; ++i) {
960               Procedure subproc = subprocs[i];
961               if (subproc == null) {
962                 String msg = "subproc[" + i + "] is null, aborting the procedure";
963                 procedure.setFailure(new RemoteProcedureException(msg,
964                   new IllegalArgumentIOException(msg)));
965                 subprocs = null;
966                 break;
967               }
968 
969               assert subproc.getState() == ProcedureState.INITIALIZING;
970               subproc.setParentProcId(procedure.getProcId());
971               subproc.setProcId(nextProcId());
972             }
973 
974             if (!procedure.isFailed()) {
975               procedure.setChildrenLatch(subprocs.length);
976               switch (procedure.getState()) {
977                 case RUNNABLE:
978                   procedure.setState(ProcedureState.WAITING);
979                   break;
980                 case WAITING_TIMEOUT:
981                   waitingTimeout.add(procedure);
982                   break;
983                 default:
984                   break;
985               }
986             }
987           }
988         } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
989           waitingTimeout.add(procedure);
990         } else {
991           // No subtask, so we are done
992           procedure.setState(ProcedureState.FINISHED);
993         }
994       }
995 
996       // Add the procedure to the stack
997       procStack.addRollbackStep(procedure);
998 
999       // allows to kill the executor before something is stored to the wal.
1000       // useful to test the procedure recovery.
1001       if (testing != null && testing.shouldKillBeforeStoreUpdate()) {
1002         if (LOG.isDebugEnabled()) {
1003           LOG.debug("TESTING: Kill before store update");
1004         }
1005         stop();
1006         return;
1007       }
1008 
1009       // Commit the transaction
1010       if (subprocs != null && !procedure.isFailed()) {
1011         if (LOG.isTraceEnabled()) {
1012           LOG.trace("Store add " + procedure + " children " + Arrays.toString(subprocs));
1013         }
1014         store.insert(procedure, subprocs);
1015       } else {
1016         if (LOG.isTraceEnabled()) {
1017           LOG.trace("Store update " + procedure);
1018         }
1019         store.update(procedure);
1020       }
1021 
1022       // if the store is not running we are aborting
1023       if (!store.isRunning()) {
1024         return;
1025       }
1026 
1027       assert (reExecute && subprocs == null) || !reExecute;
1028     } while (reExecute);
1029 
1030     // Submit the new subprocedures
1031     if (subprocs != null && !procedure.isFailed()) {
1032       for (int i = 0; i < subprocs.length; ++i) {
1033         Procedure subproc = subprocs[i];
1034         assert !procedures.containsKey(subproc.getProcId());
1035         procedures.put(subproc.getProcId(), subproc);
1036         runnables.addFront(subproc);
1037       }
1038     }
1039 
1040     if (procedure.isFinished() && procedure.hasParent()) {
1041       Procedure parent = procedures.get(procedure.getParentProcId());
1042       if (parent == null) {
1043         assert procStack.isRollingback();
1044         return;
1045       }
1046 
1047       // If this procedure is the last child awake the parent procedure
1048       if (LOG.isTraceEnabled()) {
1049         LOG.trace(parent + " child is done: " + procedure);
1050       }
1051       if (parent.childrenCountDown() && parent.getState() == ProcedureState.WAITING) {
1052         parent.setState(ProcedureState.RUNNABLE);
1053         store.update(parent);
1054         runnables.addFront(parent);
1055         if (LOG.isTraceEnabled()) {
1056           LOG.trace(parent + " all the children finished their work, resume.");
1057         }
1058         return;
1059       }
1060     }
1061   }
1062 
1063   private void sendProcedureLoadedNotification(final long procId) {
1064     if (!this.listeners.isEmpty()) {
1065       for (ProcedureExecutorListener listener: this.listeners) {
1066         try {
1067           listener.procedureLoaded(procId);
1068         } catch (Throwable e) {
1069           LOG.error("The listener " + listener + " had an error: " + e.getMessage(), e);
1070         }
1071       }
1072     }
1073   }
1074 
1075   private void sendProcedureAddedNotification(final long procId) {
1076     if (!this.listeners.isEmpty()) {
1077       for (ProcedureExecutorListener listener: this.listeners) {
1078         try {
1079           listener.procedureAdded(procId);
1080         } catch (Throwable e) {
1081           LOG.error("The listener " + listener + " had an error: " + e.getMessage(), e);
1082         }
1083       }
1084     }
1085   }
1086 
1087   private void sendProcedureFinishedNotification(final long procId) {
1088     if (!this.listeners.isEmpty()) {
1089       for (ProcedureExecutorListener listener: this.listeners) {
1090         try {
1091           listener.procedureFinished(procId);
1092         } catch (Throwable e) {
1093           LOG.error("The listener " + listener + " had an error: " + e.getMessage(), e);
1094         }
1095       }
1096     }
1097   }
1098 
1099   private long nextProcId() {
1100     long procId = lastProcId.incrementAndGet();
1101     if (procId < 0) {
1102       while (!lastProcId.compareAndSet(procId, 0)) {
1103         procId = lastProcId.get();
1104         if (procId >= 0)
1105           break;
1106       }
1107       while (procedures.containsKey(procId)) {
1108         procId = lastProcId.incrementAndGet();
1109       }
1110     }
1111     return procId;
1112   }
1113 
1114   private Long getRootProcedureId(Procedure proc) {
1115     return Procedure.getRootProcedureId(procedures, proc);
1116   }
1117 
1118   private void procedureFinished(final Procedure proc) {
1119     // call the procedure completion cleanup handler
1120     try {
1121       proc.completionCleanup(getEnvironment());
1122     } catch (Throwable e) {
1123       // Catch NullPointerExceptions or similar errors...
1124       LOG.error("CODE-BUG: uncatched runtime exception for procedure: " + proc, e);
1125     }
1126 
1127     // update the executor internal state maps
1128     completed.put(proc.getProcId(), Procedure.createProcedureInfo(proc));
1129     rollbackStack.remove(proc.getProcId());
1130     procedures.remove(proc.getProcId());
1131 
1132     // call the runnableSet completion cleanup handler
1133     try {
1134       runnables.completionCleanup(proc);
1135     } catch (Throwable e) {
1136       // Catch NullPointerExceptions or similar errors...
1137       LOG.error("CODE-BUG: uncatched runtime exception for runnableSet: " + runnables, e);
1138     }
1139 
1140     // Notify the listeners
1141     sendProcedureFinishedNotification(proc.getProcId());
1142   }
1143 
1144   public Pair<ProcedureInfo, Procedure> getResultOrProcedure(final long procId) {
1145     ProcedureInfo result = completed.get(procId);
1146     Procedure proc = null;
1147     if (result == null) {
1148       proc = procedures.get(procId);
1149       if (proc == null) {
1150         result = completed.get(procId);
1151       }
1152     }
1153     return new Pair(result, proc);
1154   }
1155 }