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