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