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