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