Improve segmented journal actor metrics
[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 com.google.common.base.Preconditions.checkArgument;
11 import static java.util.Objects.requireNonNull;
12 import static org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper.CAN_COMMIT_ERROR_MAPPER;
13 import static org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper.COMMIT_ERROR_MAPPER;
14 import static org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper.PRE_COMMIT_MAPPER;
15
16 import com.google.common.annotations.Beta;
17 import com.google.common.util.concurrent.AbstractFuture;
18 import com.google.common.util.concurrent.FluentFuture;
19 import com.google.common.util.concurrent.FutureCallback;
20 import com.google.common.util.concurrent.Futures;
21 import com.google.common.util.concurrent.MoreExecutors;
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.mdsal.common.api.CommitInfo;
27 import org.opendaylight.mdsal.common.api.DataStoreUnavailableException;
28 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
29 import org.opendaylight.mdsal.common.api.TransactionCommitFailedException;
30 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
31 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
32 import org.opendaylight.mdsal.dom.spi.AbstractDOMDataBroker;
33 import org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper;
34 import org.opendaylight.mdsal.dom.spi.store.DOMStore;
35 import org.opendaylight.mdsal.dom.spi.store.DOMStoreThreePhaseCommitCohort;
36 import org.opendaylight.yangtools.util.DurationStatisticsTracker;
37 import org.opendaylight.yangtools.yang.common.Empty;
38 import org.osgi.service.component.annotations.Activate;
39 import org.osgi.service.component.annotations.Component;
40 import org.osgi.service.component.annotations.Deactivate;
41 import org.osgi.service.component.annotations.Reference;
42 import org.slf4j.Logger;
43 import org.slf4j.LoggerFactory;
44
45 /**
46  * ConcurrentDOMDataBroker commits transactions concurrently. The 3
47  * commit phases (canCommit, preCommit, and commit) are performed serially and non-blocking
48  * (ie async) per transaction but multiple transaction commits can run concurrent.
49  *
50  * @author Thomas Pantelis
51  */
52 @Beta
53 @Component(service = DOMDataBroker.class, property = "type=default")
54 public class ConcurrentDOMDataBroker extends AbstractDOMDataBroker {
55     private static final Logger LOG = LoggerFactory.getLogger(ConcurrentDOMDataBroker.class);
56     private static final String CAN_COMMIT = "CAN_COMMIT";
57     private static final String PRE_COMMIT = "PRE_COMMIT";
58     private static final String COMMIT = "COMMIT";
59
60     private final DurationStatisticsTracker commitStatsTracker;
61
62     /**
63      * This executor is used to execute Future listener callback Runnables async.
64      */
65     private final Executor clientFutureCallbackExecutor;
66
67     public ConcurrentDOMDataBroker(final Map<LogicalDatastoreType, DOMStore> datastores,
68             final Executor listenableFutureExecutor) {
69         this(datastores, listenableFutureExecutor, DurationStatisticsTracker.createConcurrent());
70     }
71
72     public ConcurrentDOMDataBroker(final Map<LogicalDatastoreType, DOMStore> datastores,
73             final Executor listenableFutureExecutor, final DurationStatisticsTracker commitStatsTracker) {
74         super(datastores);
75         clientFutureCallbackExecutor = requireNonNull(listenableFutureExecutor);
76         this.commitStatsTracker = requireNonNull(commitStatsTracker);
77     }
78
79     @Activate
80     public ConcurrentDOMDataBroker(@Reference final DataBrokerCommitExecutor commitExecutor,
81             @Reference(target = "(type=distributed-config)") final DOMStore configDatastore,
82             @Reference(target = "(type=distributed-operational)") final DOMStore operDatastore) {
83         this(Map.of(
84             LogicalDatastoreType.CONFIGURATION, configDatastore, LogicalDatastoreType.OPERATIONAL, operDatastore),
85             commitExecutor.executor(), commitExecutor.commitStatsTracker());
86         LOG.info("DOM Data Broker started");
87     }
88
89     @Override
90     @Deactivate
91     public void close() {
92         LOG.info("DOM Data Broker stopping");
93         super.close();
94         LOG.info("DOM Data Broker stopped");
95     }
96
97     @Override
98     protected FluentFuture<? extends CommitInfo> commit(final DOMDataTreeWriteTransaction transaction,
99             final DOMStoreThreePhaseCommitCohort cohort) {
100
101         checkArgument(transaction != null, "Transaction must not be null.");
102         checkArgument(cohort != null, "Cohorts must not be null.");
103         LOG.debug("Tx: {} is submitted for execution.", transaction.getIdentifier());
104
105         final var clientSubmitFuture = new AsyncNotifyingSettableFuture(clientFutureCallbackExecutor);
106         doCanCommit(clientSubmitFuture, transaction, cohort);
107         return FluentFuture.from(clientSubmitFuture);
108     }
109
110     private void doCanCommit(final AsyncNotifyingSettableFuture clientSubmitFuture,
111             final DOMDataTreeWriteTransaction transaction,
112             final DOMStoreThreePhaseCommitCohort cohort) {
113         final long startTime = System.nanoTime();
114
115         Futures.addCallback(cohort.canCommit(), new FutureCallback<>() {
116             @Override
117             public void onSuccess(final Boolean result) {
118                 if (result == null || !result) {
119                     onFailure(new TransactionCommitFailedException("Can Commit failed, no detailed cause available."));
120                 } else {
121                     doPreCommit(startTime, clientSubmitFuture, transaction, cohort);
122                 }
123             }
124
125             @Override
126             public void onFailure(final Throwable failure) {
127                 handleException(clientSubmitFuture, transaction, cohort, CAN_COMMIT, CAN_COMMIT_ERROR_MAPPER, failure);
128             }
129         }, MoreExecutors.directExecutor());
130     }
131
132     private void doPreCommit(final long startTime, final AsyncNotifyingSettableFuture clientSubmitFuture,
133             final DOMDataTreeWriteTransaction transaction, final DOMStoreThreePhaseCommitCohort cohort) {
134         Futures.addCallback(cohort.preCommit(), new FutureCallback<>() {
135             @Override
136             public void onSuccess(final Empty result) {
137                 doCommit(startTime, clientSubmitFuture, transaction, cohort);
138             }
139
140             @Override
141             public void onFailure(final Throwable failure) {
142                 handleException(clientSubmitFuture, transaction, cohort, PRE_COMMIT, PRE_COMMIT_MAPPER, failure);
143             }
144         }, MoreExecutors.directExecutor());
145     }
146
147     private void doCommit(final long startTime, final AsyncNotifyingSettableFuture clientSubmitFuture,
148             final DOMDataTreeWriteTransaction transaction, final DOMStoreThreePhaseCommitCohort cohort) {
149         Futures.addCallback(cohort.commit(), new FutureCallback<CommitInfo>() {
150             @Override
151             public void onSuccess(final CommitInfo result) {
152                 commitStatsTracker.addDuration(System.nanoTime() - startTime);
153                 clientSubmitFuture.set();
154             }
155
156             @Override
157             public void onFailure(final Throwable throwable) {
158                 handleException(clientSubmitFuture, transaction, cohort, COMMIT, COMMIT_ERROR_MAPPER, throwable);
159             }
160         }, MoreExecutors.directExecutor());
161     }
162
163     private static void handleException(final AsyncNotifyingSettableFuture clientSubmitFuture,
164             final DOMDataTreeWriteTransaction transaction, final DOMStoreThreePhaseCommitCohort cohort,
165             final String phase, final TransactionCommitFailedExceptionMapper exMapper, final Throwable throwable) {
166         if (clientSubmitFuture.isDone()) {
167             // We must have had failures from multiple cohorts.
168             return;
169         }
170
171         // Use debug instead of warn level here because this exception gets propagate back to the caller via the Future
172         LOG.debug("Tx: {} Error during phase {}, starting Abort", transaction.getIdentifier(), phase, throwable);
173
174         // Propagate the original exception
175         final Exception e;
176         if (throwable instanceof NoShardLeaderException || throwable instanceof ShardLeaderNotRespondingException) {
177             e = new DataStoreUnavailableException(throwable.getMessage(), throwable);
178         } else if (throwable instanceof Exception ex) {
179             e = ex;
180         } else {
181             e = new RuntimeException("Unexpected error occurred", throwable);
182         }
183         clientSubmitFuture.setException(exMapper.apply(e));
184
185         // abort
186         Futures.addCallback(cohort.abort(), new FutureCallback<Empty>() {
187             @Override
188             public void onSuccess(final Empty result) {
189                 // Propagate the original exception to the client.
190                 LOG.debug("Tx: {} aborted successfully", transaction.getIdentifier());
191             }
192
193             @Override
194             public void onFailure(final Throwable failure) {
195                 LOG.error("Tx: {} Error during Abort.", transaction.getIdentifier(), failure);
196             }
197         }, MoreExecutors.directExecutor());
198     }
199
200     /**
201      * A settable future that uses an {@link Executor} to execute listener callback Runnables,
202      * registered via {@link #addListener}, asynchronously when this future completes. This is
203      * done to guarantee listener executions are off-loaded onto another thread to avoid blocking
204      * the thread that completed this future, as a common use case is to pass an executor that runs
205      * tasks in the same thread as the caller (ie MoreExecutors#sameThreadExecutor)
206      * to {@link #addListener}.
207      * FIXME: This class should probably be moved to yangtools common utils for re-usability and
208      * unified with AsyncNotifyingListenableFutureTask.
209      */
210     private static class AsyncNotifyingSettableFuture extends AbstractFuture<CommitInfo> {
211         /**
212          * ThreadLocal used to detect if the task completion thread is running the future listener Runnables.
213          */
214         private static final ThreadLocal<Boolean> ON_TASK_COMPLETION_THREAD_TL = new ThreadLocal<>();
215
216         private final Executor listenerExecutor;
217
218         AsyncNotifyingSettableFuture(final Executor listenerExecutor) {
219             this.listenerExecutor = requireNonNull(listenerExecutor);
220         }
221
222         @Override
223         public void addListener(final Runnable listener, final Executor executor) {
224             // Wrap the listener Runnable in a DelegatingRunnable. If the specified executor is one
225             // that runs tasks in the same thread as the caller submitting the task
226             // (e.g. {@link com.google.common.util.concurrent.MoreExecutors#sameThreadExecutor}) and
227             // the listener is executed from the #set methods, then the DelegatingRunnable will detect
228             // this via the ThreadLocal and submit the listener Runnable to the listenerExecutor.
229             //
230             // On the other hand, if this task is already complete, the call to ExecutionList#add in
231             // superclass will execute the listener Runnable immediately and, since the ThreadLocal
232             // won't be set, the DelegatingRunnable will run the listener Runnable inline.
233             super.addListener(new DelegatingRunnable(listener, listenerExecutor), executor);
234         }
235
236         boolean set() {
237             ON_TASK_COMPLETION_THREAD_TL.set(Boolean.TRUE);
238             try {
239                 return super.set(CommitInfo.empty());
240             } finally {
241                 ON_TASK_COMPLETION_THREAD_TL.set(null);
242             }
243         }
244
245         @Override
246         protected boolean setException(final Throwable throwable) {
247             ON_TASK_COMPLETION_THREAD_TL.set(Boolean.TRUE);
248             try {
249                 return super.setException(throwable);
250             } finally {
251                 ON_TASK_COMPLETION_THREAD_TL.set(null);
252             }
253         }
254
255         private static final class DelegatingRunnable implements Runnable {
256             private final Runnable delegate;
257             private final Executor executor;
258
259             DelegatingRunnable(final Runnable delegate, final Executor executor) {
260                 this.delegate = requireNonNull(delegate);
261                 this.executor = requireNonNull(executor);
262             }
263
264             @Override
265             public void run() {
266                 if (ON_TASK_COMPLETION_THREAD_TL.get() != null) {
267                     // We're running on the task completion thread so off-load to the executor.
268                     LOG.trace("Submitting ListenenableFuture Runnable from thread {} to executor {}",
269                             Thread.currentThread().getName(), executor);
270                     executor.execute(delegate);
271                 } else {
272                     // We're not running on the task completion thread so run the delegate inline.
273                     LOG.trace("Executing ListenenableFuture Runnable on this thread: {}",
274                             Thread.currentThread().getName());
275                     delegate.run();
276                 }
277             }
278         }
279     }
280
281     @Override
282     public String toString() {
283         return "Clustered ConcurrentDOMDataBroker";
284     }
285 }