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