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