Bug 2347: DOMConcurrentDataCommitCoordinator uses wrong phase name
[controller.git] / opendaylight / md-sal / sal-dom-broker / src / main / java / org / opendaylight / controller / md / sal / dom / broker / impl / DOMConcurrentDataCommitCoordinator.java
1 /*
2  * Copyright (c) 2014 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.md.sal.dom.broker.impl;
9
10 import com.google.common.base.Preconditions;
11 import com.google.common.collect.Iterables;
12 import com.google.common.util.concurrent.AbstractFuture;
13 import com.google.common.util.concurrent.AbstractListeningExecutorService;
14 import com.google.common.util.concurrent.CheckedFuture;
15 import com.google.common.util.concurrent.FutureCallback;
16 import com.google.common.util.concurrent.Futures;
17 import com.google.common.util.concurrent.ListenableFuture;
18 import java.util.List;
19 import java.util.concurrent.Executor;
20 import java.util.concurrent.ExecutorService;
21 import java.util.concurrent.TimeUnit;
22 import java.util.concurrent.atomic.AtomicInteger;
23 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
24 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
25 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
26 import org.opendaylight.yangtools.util.DurationStatisticsTracker;
27 import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture;
28 import org.slf4j.Logger;
29 import org.slf4j.LoggerFactory;
30
31 /**
32  * Implementation of DOMDataCommitExecutor that coordinates transaction commits concurrently. The 3
33  * commit phases (canCommit, preCommit, and commit) are performed serially and non-blocking
34  * (ie async) per transaction but multiple transaction commits can run concurrent.
35  *
36  * @author Thomas Pantelis
37  */
38 public class DOMConcurrentDataCommitCoordinator implements DOMDataCommitExecutor {
39
40     private static final String CAN_COMMIT = "CAN_COMMIT";
41     private static final String PRE_COMMIT = "PRE_COMMIT";
42     private static final String COMMIT = "COMMIT";
43
44     private static final Logger LOG = LoggerFactory.getLogger(DOMConcurrentDataCommitCoordinator.class);
45
46     private final DurationStatisticsTracker commitStatsTracker = DurationStatisticsTracker.createConcurrent();
47
48     /**
49      * This executor is used to execute Future listener callback Runnables async.
50      */
51     private final ExecutorService clientFutureCallbackExecutor;
52
53     /**
54      * This executor is re-used internally in calls to Futures#addCallback to avoid the overhead
55      * of Futures#addCallback creating a MoreExecutors#sameThreadExecutor for each call.
56      */
57     private final ExecutorService internalFutureCallbackExecutor = new SimpleSameThreadExecutor();
58
59     public DOMConcurrentDataCommitCoordinator(ExecutorService listenableFutureExecutor) {
60         this.clientFutureCallbackExecutor = Preconditions.checkNotNull(listenableFutureExecutor);
61     }
62
63     public DurationStatisticsTracker getCommitStatsTracker() {
64         return commitStatsTracker;
65     }
66
67     @Override
68     public CheckedFuture<Void, TransactionCommitFailedException> submit(DOMDataWriteTransaction transaction,
69             Iterable<DOMStoreThreePhaseCommitCohort> cohorts) {
70
71         Preconditions.checkArgument(transaction != null, "Transaction must not be null.");
72         Preconditions.checkArgument(cohorts != null, "Cohorts must not be null.");
73         LOG.debug("Tx: {} is submitted for execution.", transaction.getIdentifier());
74
75         final int cohortSize = Iterables.size(cohorts);
76         final AsyncNotifyingSettableFuture clientSubmitFuture =
77                 new AsyncNotifyingSettableFuture(clientFutureCallbackExecutor);
78
79         doCanCommit(clientSubmitFuture, transaction, cohorts, cohortSize);
80
81         return MappingCheckedFuture.create(clientSubmitFuture,
82                 TransactionCommitFailedExceptionMapper.COMMIT_ERROR_MAPPER);
83     }
84
85     private void doCanCommit(final AsyncNotifyingSettableFuture clientSubmitFuture,
86             final DOMDataWriteTransaction transaction,
87             final Iterable<DOMStoreThreePhaseCommitCohort> cohorts, final int cohortSize) {
88
89         final long startTime = System.nanoTime();
90
91         // Not using Futures.allAsList here to avoid its internal overhead.
92         final AtomicInteger remaining = new AtomicInteger(cohortSize);
93         FutureCallback<Boolean> futureCallback = new FutureCallback<Boolean>() {
94             @Override
95             public void onSuccess(Boolean result) {
96                 if (result == null || !result) {
97                     handleException(clientSubmitFuture, transaction, cohorts, cohortSize,
98                             CAN_COMMIT, new TransactionCommitFailedException(
99                                             "Can Commit failed, no detailed cause available."));
100                 } else {
101                     if(remaining.decrementAndGet() == 0) {
102                         // All cohorts completed successfully - we can move on to the preCommit phase
103                         doPreCommit(startTime, clientSubmitFuture, transaction, cohorts, cohortSize);
104                     }
105                 }
106             }
107
108             @Override
109             public void onFailure(Throwable t) {
110                 handleException(clientSubmitFuture, transaction, cohorts, cohortSize, CAN_COMMIT, t);
111             }
112         };
113
114         for(DOMStoreThreePhaseCommitCohort cohort: cohorts) {
115             ListenableFuture<Boolean> canCommitFuture = cohort.canCommit();
116             Futures.addCallback(canCommitFuture, futureCallback, internalFutureCallbackExecutor);
117         }
118     }
119
120     private void doPreCommit(final long startTime, final AsyncNotifyingSettableFuture clientSubmitFuture,
121             final DOMDataWriteTransaction transaction,
122             final Iterable<DOMStoreThreePhaseCommitCohort> cohorts, final int cohortSize) {
123
124         // Not using Futures.allAsList here to avoid its internal overhead.
125         final AtomicInteger remaining = new AtomicInteger(cohortSize);
126         FutureCallback<Void> futureCallback = new FutureCallback<Void>() {
127             @Override
128             public void onSuccess(Void notUsed) {
129                 if(remaining.decrementAndGet() == 0) {
130                     // All cohorts completed successfully - we can move on to the commit phase
131                     doCommit(startTime, clientSubmitFuture, transaction, cohorts, cohortSize);
132                 }
133             }
134
135             @Override
136             public void onFailure(Throwable t) {
137                 handleException(clientSubmitFuture, transaction, cohorts, cohortSize, PRE_COMMIT, t);
138             }
139         };
140
141         for(DOMStoreThreePhaseCommitCohort cohort: cohorts) {
142             ListenableFuture<Void> preCommitFuture = cohort.preCommit();
143             Futures.addCallback(preCommitFuture, futureCallback, internalFutureCallbackExecutor);
144         }
145     }
146
147     private void doCommit(final long startTime, final AsyncNotifyingSettableFuture clientSubmitFuture,
148             final DOMDataWriteTransaction transaction,
149             final Iterable<DOMStoreThreePhaseCommitCohort> cohorts, final int cohortSize) {
150
151         // Not using Futures.allAsList here to avoid its internal overhead.
152         final AtomicInteger remaining = new AtomicInteger(cohortSize);
153         FutureCallback<Void> futureCallback = new FutureCallback<Void>() {
154             @Override
155             public void onSuccess(Void notUsed) {
156                 if(remaining.decrementAndGet() == 0) {
157                     // All cohorts completed successfully - we're done.
158                     commitStatsTracker.addDuration(System.nanoTime() - startTime);
159
160                     clientSubmitFuture.set();
161                 }
162             }
163
164             @Override
165             public void onFailure(Throwable t) {
166                 handleException(clientSubmitFuture, transaction, cohorts, cohortSize, COMMIT, t);
167             }
168         };
169
170         for(DOMStoreThreePhaseCommitCohort cohort: cohorts) {
171             ListenableFuture<Void> commitFuture = cohort.commit();
172             Futures.addCallback(commitFuture, futureCallback, internalFutureCallbackExecutor);
173         }
174     }
175
176     private void handleException(final AsyncNotifyingSettableFuture clientSubmitFuture,
177             final DOMDataWriteTransaction transaction,
178             final Iterable<DOMStoreThreePhaseCommitCohort> cohorts, int cohortSize,
179             final String phase, final Throwable t) {
180
181         if(clientSubmitFuture.isDone()) {
182             // We must have had failures from multiple cohorts.
183             return;
184         }
185
186         LOG.warn("Tx: {} Error during phase {}, starting Abort", transaction.getIdentifier(), phase, t);
187         Exception e;
188         if(t instanceof Exception) {
189             e = (Exception)t;
190         } else {
191             e = new RuntimeException("Unexpected error occurred", t);
192         }
193
194         final TransactionCommitFailedException clientException =
195                 TransactionCommitFailedExceptionMapper.CAN_COMMIT_ERROR_MAPPER.apply(e);
196
197         // Transaction failed - tell all cohorts to abort.
198
199         @SuppressWarnings("unchecked")
200         ListenableFuture<Void>[] canCommitFutures = new ListenableFuture[cohortSize];
201         int i = 0;
202         for(DOMStoreThreePhaseCommitCohort cohort: cohorts) {
203             canCommitFutures[i++] = cohort.abort();
204         }
205
206         ListenableFuture<List<Void>> combinedFuture = Futures.allAsList(canCommitFutures);
207         Futures.addCallback(combinedFuture, new FutureCallback<List<Void>>() {
208             @Override
209             public void onSuccess(List<Void> notUsed) {
210                 // Propagate the original exception to the client.
211                 clientSubmitFuture.setException(clientException);
212             }
213
214             @Override
215             public void onFailure(Throwable t) {
216                 LOG.error("Tx: {} Error during Abort.", transaction.getIdentifier(), t);
217
218                 // Propagate the original exception as that is what caused the Tx to fail and is
219                 // what's interesting to the client.
220                 clientSubmitFuture.setException(clientException);
221             }
222         }, internalFutureCallbackExecutor);
223     }
224
225     /**
226      * A settable future that uses an {@link Executor} to execute listener callback Runnables,
227      * registered via {@link #addListener}, asynchronously when this future completes. This is
228      * done to guarantee listener executions are off-loaded onto another thread to avoid blocking
229      * the thread that completed this future, as a common use case is to pass an executor that runs
230      * tasks in the same thread as the caller (ie MoreExecutors#sameThreadExecutor)
231      * to {@link #addListener}.
232      *
233      * FIXME: This class should probably be moved to yangtools common utils for re-usability and
234      * unified with AsyncNotifyingListenableFutureTask.
235      */
236     private static class AsyncNotifyingSettableFuture extends AbstractFuture<Void> {
237
238         /**
239          * ThreadLocal used to detect if the task completion thread is running the future listener Runnables.
240          */
241         private static final ThreadLocal<Boolean> ON_TASK_COMPLETION_THREAD_TL = new ThreadLocal<Boolean>();
242
243         private final ExecutorService listenerExecutor;
244
245         AsyncNotifyingSettableFuture(ExecutorService listenerExecutor) {
246             this.listenerExecutor = listenerExecutor;
247         }
248
249         @Override
250         public void addListener(final Runnable listener, final Executor executor) {
251             // Wrap the listener Runnable in a DelegatingRunnable. If the specified executor is one
252             // that runs tasks in the same thread as the caller submitting the task
253             // (e.g. {@link com.google.common.util.concurrent.MoreExecutors#sameThreadExecutor}) and
254             // the listener is executed from the #set methods, then the DelegatingRunnable will detect
255             // this via the ThreadLocal and submit the listener Runnable to the listenerExecutor.
256             //
257             // On the other hand, if this task is already complete, the call to ExecutionList#add in
258             // superclass will execute the listener Runnable immediately and, since the ThreadLocal
259             // won't be set, the DelegatingRunnable will run the listener Runnable inline.
260             super.addListener(new DelegatingRunnable(listener, listenerExecutor), executor);
261         }
262
263         boolean set() {
264             ON_TASK_COMPLETION_THREAD_TL.set(Boolean.TRUE);
265             try {
266                 return super.set(null);
267             } finally {
268                 ON_TASK_COMPLETION_THREAD_TL.set(null);
269             }
270         }
271
272         @Override
273         protected boolean setException(Throwable throwable) {
274             ON_TASK_COMPLETION_THREAD_TL.set(Boolean.TRUE);
275             try {
276                 return super.setException(throwable);
277             } finally {
278                 ON_TASK_COMPLETION_THREAD_TL.set(null);
279             }
280         }
281
282         private static final class DelegatingRunnable implements Runnable {
283             private final Runnable delegate;
284             private final Executor executor;
285
286             DelegatingRunnable(final Runnable delegate, final Executor executor) {
287                 this.delegate = Preconditions.checkNotNull(delegate);
288                 this.executor = Preconditions.checkNotNull(executor);
289             }
290
291             @Override
292             public void run() {
293                 if (ON_TASK_COMPLETION_THREAD_TL.get() != null) {
294                     // We're running on the task completion thread so off-load to the executor.
295                     LOG.trace("Submitting ListenenableFuture Runnable from thread {} to executor {}",
296                             Thread.currentThread().getName(), executor);
297                     executor.execute(delegate);
298                 } else {
299                     // We're not running on the task completion thread so run the delegate inline.
300                     LOG.trace("Executing ListenenableFuture Runnable on this thread: {}",
301                             Thread.currentThread().getName());
302                     delegate.run();
303                 }
304             }
305         }
306     }
307
308     /**
309      * A simple same-thread executor without the internal locking overhead that
310      * MoreExecutors#sameThreadExecutor has. The #execute method is the only one of concern - we
311      * don't shutdown the executor so the other methods irrelevant.
312      */
313     private static class SimpleSameThreadExecutor extends AbstractListeningExecutorService {
314
315         @Override
316         public void execute(Runnable command) {
317             command.run();
318         }
319
320         @Override
321         public boolean awaitTermination(long arg0, TimeUnit arg1) throws InterruptedException {
322             return true;
323         }
324
325         @Override
326         public boolean isShutdown() {
327             return false;
328         }
329
330         @Override
331         public boolean isTerminated() {
332             return false;
333         }
334
335         @Override
336         public void shutdown() {
337         }
338
339         @Override
340         public List<Runnable> shutdownNow() {
341             return null;
342         }
343     }
344 }