Improve segmented journal actor metrics
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / databroker / ConcurrentDOMDataBroker.java
index 1199fbf7bca4a4c5f60485690b4e82eb37ca062f..d6da8487e0f4af80278028d48bfb660f08d6cf0c 100644 (file)
@@ -7,22 +7,18 @@
  */
 package org.opendaylight.controller.cluster.databroker;
 
-import static org.opendaylight.mdsal.dom.broker.TransactionCommitFailedExceptionMapper.CAN_COMMIT_ERROR_MAPPER;
-import static org.opendaylight.mdsal.dom.broker.TransactionCommitFailedExceptionMapper.COMMIT_ERROR_MAPPER;
-import static org.opendaylight.mdsal.dom.broker.TransactionCommitFailedExceptionMapper.PRE_COMMIT_MAPPER;
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+import static org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper.CAN_COMMIT_ERROR_MAPPER;
+import static org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper.COMMIT_ERROR_MAPPER;
+import static org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper.PRE_COMMIT_MAPPER;
 
 import com.google.common.annotations.Beta;
-import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.AbstractFuture;
 import com.google.common.util.concurrent.FluentFuture;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Executor;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
@@ -31,11 +27,18 @@ import org.opendaylight.mdsal.common.api.CommitInfo;
 import org.opendaylight.mdsal.common.api.DataStoreUnavailableException;
 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
 import org.opendaylight.mdsal.common.api.TransactionCommitFailedException;
+import org.opendaylight.mdsal.dom.api.DOMDataBroker;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
-import org.opendaylight.mdsal.dom.broker.TransactionCommitFailedExceptionMapper;
+import org.opendaylight.mdsal.dom.spi.AbstractDOMDataBroker;
+import org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper;
 import org.opendaylight.mdsal.dom.spi.store.DOMStore;
 import org.opendaylight.mdsal.dom.spi.store.DOMStoreThreePhaseCommitCohort;
 import org.opendaylight.yangtools.util.DurationStatisticsTracker;
+import org.opendaylight.yangtools.yang.common.Empty;
+import org.osgi.service.component.annotations.Activate;
+import org.osgi.service.component.annotations.Component;
+import org.osgi.service.component.annotations.Deactivate;
+import org.osgi.service.component.annotations.Reference;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,7 +50,8 @@ import org.slf4j.LoggerFactory;
  * @author Thomas Pantelis
  */
 @Beta
-public class ConcurrentDOMDataBroker extends AbstractDOMBroker {
+@Component(service = DOMDataBroker.class, property = "type=default")
+public class ConcurrentDOMDataBroker extends AbstractDOMDataBroker {
     private static final Logger LOG = LoggerFactory.getLogger(ConcurrentDOMDataBroker.class);
     private static final String CAN_COMMIT = "CAN_COMMIT";
     private static final String PRE_COMMIT = "PRE_COMMIT";
@@ -68,138 +72,97 @@ public class ConcurrentDOMDataBroker extends AbstractDOMBroker {
     public ConcurrentDOMDataBroker(final Map<LogicalDatastoreType, DOMStore> datastores,
             final Executor listenableFutureExecutor, final DurationStatisticsTracker commitStatsTracker) {
         super(datastores);
-        this.clientFutureCallbackExecutor = Preconditions.checkNotNull(listenableFutureExecutor);
-        this.commitStatsTracker = Preconditions.checkNotNull(commitStatsTracker);
+        clientFutureCallbackExecutor = requireNonNull(listenableFutureExecutor);
+        this.commitStatsTracker = requireNonNull(commitStatsTracker);
     }
 
-    public DurationStatisticsTracker getCommitStatsTracker() {
-        return commitStatsTracker;
+    @Activate
+    public ConcurrentDOMDataBroker(@Reference final DataBrokerCommitExecutor commitExecutor,
+            @Reference(target = "(type=distributed-config)") final DOMStore configDatastore,
+            @Reference(target = "(type=distributed-operational)") final DOMStore operDatastore) {
+        this(Map.of(
+            LogicalDatastoreType.CONFIGURATION, configDatastore, LogicalDatastoreType.OPERATIONAL, operDatastore),
+            commitExecutor.executor(), commitExecutor.commitStatsTracker());
+        LOG.info("DOM Data Broker started");
     }
 
     @Override
-    protected FluentFuture<? extends CommitInfo> commit(
-            final DOMDataTreeWriteTransaction transaction, final Collection<DOMStoreThreePhaseCommitCohort> cohorts) {
-
-        Preconditions.checkArgument(transaction != null, "Transaction must not be null.");
-        Preconditions.checkArgument(cohorts != null, "Cohorts must not be null.");
-        LOG.debug("Tx: {} is submitted for execution.", transaction.getIdentifier());
-
-        if (cohorts.isEmpty()) {
-            return CommitInfo.emptyFluentFuture();
-        }
+    @Deactivate
+    public void close() {
+        LOG.info("DOM Data Broker stopping");
+        super.close();
+        LOG.info("DOM Data Broker stopped");
+    }
 
-        final AsyncNotifyingSettableFuture clientSubmitFuture =
-                new AsyncNotifyingSettableFuture(clientFutureCallbackExecutor);
+    @Override
+    protected FluentFuture<? extends CommitInfo> commit(final DOMDataTreeWriteTransaction transaction,
+            final DOMStoreThreePhaseCommitCohort cohort) {
 
-        doCanCommit(clientSubmitFuture, transaction, cohorts);
+        checkArgument(transaction != null, "Transaction must not be null.");
+        checkArgument(cohort != null, "Cohorts must not be null.");
+        LOG.debug("Tx: {} is submitted for execution.", transaction.getIdentifier());
 
-        return FluentFuture.from(clientSubmitFuture).transform(ignored -> CommitInfo.empty(),
-                MoreExecutors.directExecutor());
+        final var clientSubmitFuture = new AsyncNotifyingSettableFuture(clientFutureCallbackExecutor);
+        doCanCommit(clientSubmitFuture, transaction, cohort);
+        return FluentFuture.from(clientSubmitFuture);
     }
 
     private void doCanCommit(final AsyncNotifyingSettableFuture clientSubmitFuture,
             final DOMDataTreeWriteTransaction transaction,
-            final Collection<DOMStoreThreePhaseCommitCohort> cohorts) {
-
+            final DOMStoreThreePhaseCommitCohort cohort) {
         final long startTime = System.nanoTime();
 
-        final Iterator<DOMStoreThreePhaseCommitCohort> cohortIterator = cohorts.iterator();
-
-        // Not using Futures.allAsList here to avoid its internal overhead.
-        FutureCallback<Boolean> futureCallback = new FutureCallback<Boolean>() {
+        Futures.addCallback(cohort.canCommit(), new FutureCallback<>() {
             @Override
             public void onSuccess(final Boolean result) {
                 if (result == null || !result) {
-                    handleException(clientSubmitFuture, transaction, cohorts, CAN_COMMIT, CAN_COMMIT_ERROR_MAPPER,
-                            new TransactionCommitFailedException("Can Commit failed, no detailed cause available."));
-                } else if (!cohortIterator.hasNext()) {
-                    // All cohorts completed successfully - we can move on to the preCommit phase
-                    doPreCommit(startTime, clientSubmitFuture, transaction, cohorts);
+                    onFailure(new TransactionCommitFailedException("Can Commit failed, no detailed cause available."));
                 } else {
-                    Futures.addCallback(cohortIterator.next().canCommit(), this, MoreExecutors.directExecutor());
+                    doPreCommit(startTime, clientSubmitFuture, transaction, cohort);
                 }
             }
 
             @Override
             public void onFailure(final Throwable failure) {
-                handleException(clientSubmitFuture, transaction, cohorts, CAN_COMMIT, CAN_COMMIT_ERROR_MAPPER, failure);
+                handleException(clientSubmitFuture, transaction, cohort, CAN_COMMIT, CAN_COMMIT_ERROR_MAPPER, failure);
             }
-        };
-
-        ListenableFuture<Boolean> canCommitFuture = cohortIterator.next().canCommit();
-        Futures.addCallback(canCommitFuture, futureCallback, MoreExecutors.directExecutor());
+        }, MoreExecutors.directExecutor());
     }
 
     private void doPreCommit(final long startTime, final AsyncNotifyingSettableFuture clientSubmitFuture,
-            final DOMDataTreeWriteTransaction transaction,
-            final Collection<DOMStoreThreePhaseCommitCohort> cohorts) {
-
-        final Iterator<DOMStoreThreePhaseCommitCohort> cohortIterator = cohorts.iterator();
-
-        // Not using Futures.allAsList here to avoid its internal overhead.
-        FutureCallback<Void> futureCallback = new FutureCallback<Void>() {
+            final DOMDataTreeWriteTransaction transaction, final DOMStoreThreePhaseCommitCohort cohort) {
+        Futures.addCallback(cohort.preCommit(), new FutureCallback<>() {
             @Override
-            public void onSuccess(final Void notUsed) {
-                if (!cohortIterator.hasNext()) {
-                    // All cohorts completed successfully - we can move on to the commit phase
-                    doCommit(startTime, clientSubmitFuture, transaction, cohorts);
-                } else {
-                    ListenableFuture<Void> preCommitFuture = cohortIterator.next().preCommit();
-                    Futures.addCallback(preCommitFuture, this, MoreExecutors.directExecutor());
-                }
+            public void onSuccess(final Empty result) {
+                doCommit(startTime, clientSubmitFuture, transaction, cohort);
             }
 
             @Override
             public void onFailure(final Throwable failure) {
-                handleException(clientSubmitFuture, transaction, cohorts, PRE_COMMIT, PRE_COMMIT_MAPPER, failure);
+                handleException(clientSubmitFuture, transaction, cohort, PRE_COMMIT, PRE_COMMIT_MAPPER, failure);
             }
-        };
-
-        ListenableFuture<Void> preCommitFuture = cohortIterator.next().preCommit();
-        Futures.addCallback(preCommitFuture, futureCallback, MoreExecutors.directExecutor());
+        }, MoreExecutors.directExecutor());
     }
 
     private void doCommit(final long startTime, final AsyncNotifyingSettableFuture clientSubmitFuture,
-            final DOMDataTreeWriteTransaction transaction,
-            final Collection<DOMStoreThreePhaseCommitCohort> cohorts) {
-
-        final Iterator<DOMStoreThreePhaseCommitCohort> cohortIterator = cohorts.iterator();
-
-        // Not using Futures.allAsList here to avoid its internal overhead.
-        FutureCallback<Void> futureCallback = new FutureCallback<Void>() {
+            final DOMDataTreeWriteTransaction transaction, final DOMStoreThreePhaseCommitCohort cohort) {
+        Futures.addCallback(cohort.commit(), new FutureCallback<CommitInfo>() {
             @Override
-            public void onSuccess(final Void notUsed) {
-                if (!cohortIterator.hasNext()) {
-                    // All cohorts completed successfully - we're done.
-                    commitStatsTracker.addDuration(System.nanoTime() - startTime);
-
-                    clientSubmitFuture.set();
-                } else {
-                    ListenableFuture<Void> commitFuture = cohortIterator.next().commit();
-                    Futures.addCallback(commitFuture, this, MoreExecutors.directExecutor());
-                }
+            public void onSuccess(final CommitInfo result) {
+                commitStatsTracker.addDuration(System.nanoTime() - startTime);
+                clientSubmitFuture.set();
             }
 
             @Override
             public void onFailure(final Throwable throwable) {
-                handleException(clientSubmitFuture, transaction, cohorts, COMMIT, COMMIT_ERROR_MAPPER, throwable);
+                handleException(clientSubmitFuture, transaction, cohort, COMMIT, COMMIT_ERROR_MAPPER, throwable);
             }
-        };
-
-        ListenableFuture<Void> commitFuture = cohortIterator.next().commit();
-        Futures.addCallback(commitFuture, futureCallback, MoreExecutors.directExecutor());
+        }, MoreExecutors.directExecutor());
     }
 
-    @SuppressFBWarnings(value = "BC_UNCONFIRMED_CAST_OF_RETURN_VALUE",
-            justification = "Pertains to the assignment of the 'clientException' var. FindBugs flags this as an "
-                + "uncomfirmed cast but the generic type in TransactionCommitFailedExceptionMapper is "
-                + "TransactionCommitFailedException and thus should be deemed as confirmed.")
     private static void handleException(final AsyncNotifyingSettableFuture clientSubmitFuture,
-            final DOMDataTreeWriteTransaction transaction,
-            final Collection<DOMStoreThreePhaseCommitCohort> cohorts,
-            final String phase, final TransactionCommitFailedExceptionMapper exMapper,
-            final Throwable throwable) {
-
+            final DOMDataTreeWriteTransaction transaction, final DOMStoreThreePhaseCommitCohort cohort,
+            final String phase, final TransactionCommitFailedExceptionMapper exMapper, final Throwable throwable) {
         if (clientSubmitFuture.isDone()) {
             // We must have had failures from multiple cohorts.
             return;
@@ -208,29 +171,21 @@ public class ConcurrentDOMDataBroker extends AbstractDOMBroker {
         // Use debug instead of warn level here because this exception gets propagate back to the caller via the Future
         LOG.debug("Tx: {} Error during phase {}, starting Abort", transaction.getIdentifier(), phase, throwable);
 
-        // Transaction failed - tell all cohorts to abort.
-        @SuppressWarnings("unchecked")
-        ListenableFuture<Void>[] canCommitFutures = new ListenableFuture[cohorts.size()];
-        int index = 0;
-        for (DOMStoreThreePhaseCommitCohort cohort : cohorts) {
-            canCommitFutures[index++] = cohort.abort();
-        }
-
         // Propagate the original exception
         final Exception e;
         if (throwable instanceof NoShardLeaderException || throwable instanceof ShardLeaderNotRespondingException) {
             e = new DataStoreUnavailableException(throwable.getMessage(), throwable);
-        } else if (throwable instanceof Exception) {
-            e = (Exception)throwable;
+        } else if (throwable instanceof Exception ex) {
+            e = ex;
         } else {
             e = new RuntimeException("Unexpected error occurred", throwable);
         }
         clientSubmitFuture.setException(exMapper.apply(e));
 
-        ListenableFuture<List<Void>> combinedFuture = Futures.allAsList(canCommitFutures);
-        Futures.addCallback(combinedFuture, new FutureCallback<List<Void>>() {
+        // abort
+        Futures.addCallback(cohort.abort(), new FutureCallback<Empty>() {
             @Override
-            public void onSuccess(final List<Void> notUsed) {
+            public void onSuccess(final Empty result) {
                 // Propagate the original exception to the client.
                 LOG.debug("Tx: {} aborted successfully", transaction.getIdentifier());
             }
@@ -252,8 +207,7 @@ public class ConcurrentDOMDataBroker extends AbstractDOMBroker {
      * FIXME: This class should probably be moved to yangtools common utils for re-usability and
      * unified with AsyncNotifyingListenableFutureTask.
      */
-    private static class AsyncNotifyingSettableFuture extends AbstractFuture<Void> {
-
+    private static class AsyncNotifyingSettableFuture extends AbstractFuture<CommitInfo> {
         /**
          * ThreadLocal used to detect if the task completion thread is running the future listener Runnables.
          */
@@ -262,7 +216,7 @@ public class ConcurrentDOMDataBroker extends AbstractDOMBroker {
         private final Executor listenerExecutor;
 
         AsyncNotifyingSettableFuture(final Executor listenerExecutor) {
-            this.listenerExecutor = Preconditions.checkNotNull(listenerExecutor);
+            this.listenerExecutor = requireNonNull(listenerExecutor);
         }
 
         @Override
@@ -282,7 +236,7 @@ public class ConcurrentDOMDataBroker extends AbstractDOMBroker {
         boolean set() {
             ON_TASK_COMPLETION_THREAD_TL.set(Boolean.TRUE);
             try {
-                return super.set(null);
+                return super.set(CommitInfo.empty());
             } finally {
                 ON_TASK_COMPLETION_THREAD_TL.set(null);
             }
@@ -303,8 +257,8 @@ public class ConcurrentDOMDataBroker extends AbstractDOMBroker {
             private final Executor executor;
 
             DelegatingRunnable(final Runnable delegate, final Executor executor) {
-                this.delegate = Preconditions.checkNotNull(delegate);
-                this.executor = Preconditions.checkNotNull(executor);
+                this.delegate = requireNonNull(delegate);
+                this.executor = requireNonNull(executor);
             }
 
             @Override