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