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