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