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