1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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
60
61
62
63
64
65
66
67
68
69
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
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
114
115
116
117
118
119
120
121
122
123
124
125
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;
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;
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;
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
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
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
209
210
211
212 private final ConcurrentHashMap<Long, ProcedureInfo> completed =
213 new ConcurrentHashMap<Long, ProcedureInfo>();
214
215
216
217
218
219
220 private final ConcurrentHashMap<Long, RootProcedureState> rollbackStack =
221 new ConcurrentHashMap<Long, RootProcedureState>();
222
223
224
225
226
227 private final ConcurrentHashMap<Long, Procedure> procedures =
228 new ConcurrentHashMap<Long, Procedure>();
229
230
231
232
233
234 private final TimeoutBlockingQueue<Procedure> waitingTimeout =
235 new TimeoutBlockingQueue<Procedure>(new ProcedureTimeoutRetriever());
236
237
238
239
240 private final ProcedureRunnableSet runnables;
241
242
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
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
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
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
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
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
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
387 if (runnableSet != null) {
388
389
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
407
408 threads = new Thread[numThreads + 1];
409 LOG.info("Starting procedure executor threads=" + threads.length);
410
411
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
422 threads[numThreads] = new Thread("ProcedureExecutorTimeout") {
423 @Override
424 public void run() {
425 timeoutLoop();
426 }
427 };
428
429
430 store.recoverLease();
431
432
433
434
435
436
437 load();
438
439
440 for (int i = 0; i < threads.length; ++i) {
441 threads[i].start();
442 }
443
444
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
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
514
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
524
525
526
527 procedureLists.add(e.getValue());
528 }
529 return procedureLists;
530 }
531
532
533
534
535
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
544 proc.setProcId(nextProcId());
545
546
547 store.insert(proc, null);
548 if (LOG.isDebugEnabled()) {
549 LOG.debug("Procedure " + proc + " added to the store.");
550 }
551
552
553 RootProcedureState stack = new RootProcedureState();
554 rollbackStack.put(proc.getProcId(), stack);
555
556
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
570
571
572
573
574
575 public boolean isFinished(final long procId) {
576 return completed.containsKey(procId);
577 }
578
579
580
581
582
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
594
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
607 result.setClientAckTime(EnvironmentEdgeManager.currentTime());
608 }
609
610
611
612
613
614
615
616 public boolean abort(final long procId) {
617 return abort(procId, true);
618 }
619
620
621
622
623
624
625
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
641
642
643
644
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
658
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
678
679
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
704 executeRollback(proc);
705 return;
706 }
707
708 RootProcedureState procStack = rollbackStack.get(rootProcId);
709 if (procStack == null) return;
710
711 do {
712
713 if (!procStack.acquire(proc)) {
714 if (procStack.setRollback()) {
715
716 if (!executeRollback(rootProcId, procStack)) {
717 procStack.unsetRollback();
718 runnables.yield(proc);
719 }
720 } else {
721
722
723
724 if (!proc.wasExecuted()) {
725 if (!executeRollback(proc)) {
726 runnables.yield(proc);
727 }
728 }
729 }
730 break;
731 }
732
733
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
744
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
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
770
771 waitingTimeout.add(proc);
772 continue;
773 }
774
775
776
777
778
779
780
781
782
783
784
785
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
798
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
812
813
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
834
835 return false;
836 }
837
838 boolean abortRollback = !executeRollback(proc);
839 abortRollback |= !isRunning() || !store.isRunning();
840
841
842
843
844 reuseLock = stackTail > 0 && (subprocStack.get(stackTail - 1) == proc) && !abortRollback;
845 if (!reuseLock) {
846 proc.releaseLock(getEnvironment());
847 }
848
849
850
851 if (abortRollback) {
852 return false;
853 }
854
855 subprocStack.remove(stackTail);
856 }
857
858
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
868
869
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
881 LOG.fatal("CODE-BUG: Uncatched runtime exception for procedure: " + proc, e);
882 }
883
884
885
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
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925 private void execProcedure(final RootProcedureState procStack, final Procedure procedure) {
926 Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE);
927
928
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
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
955 subprocs = null;
956 reExecute = true;
957 } else {
958
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
992 procedure.setState(ProcedureState.FINISHED);
993 }
994 }
995
996
997 procStack.addRollbackStep(procedure);
998
999
1000
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
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
1023 if (!store.isRunning()) {
1024 return;
1025 }
1026
1027 assert (reExecute && subprocs == null) || !reExecute;
1028 } while (reExecute);
1029
1030
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
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
1120 try {
1121 proc.completionCleanup(getEnvironment());
1122 } catch (Throwable e) {
1123
1124 LOG.error("CODE-BUG: uncatched runtime exception for procedure: " + proc, e);
1125 }
1126
1127
1128 completed.put(proc.getProcId(), Procedure.createProcedureInfo(proc));
1129 rollbackStack.remove(proc.getProcId());
1130 procedures.remove(proc.getProcId());
1131
1132
1133 try {
1134 runnables.completionCleanup(proc);
1135 } catch (Throwable e) {
1136
1137 LOG.error("CODE-BUG: uncatched runtime exception for runnableSet: " + runnables, e);
1138 }
1139
1140
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 }