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