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