Merge "Mark DomBrokerImplModule.createLegacyDataService() as deprecated"
[controller.git] / opendaylight / md-sal / sal-inmemory-datastore / src / main / java / org / opendaylight / controller / md / sal / dom / store / impl / InMemoryDOMDataStore.java
index 3e748618169889cdfcf0b9f8eb66c73e65cb4338..deddd6938ae477eeebbd9fdeb50c79b6d3694eab 100644 (file)
@@ -8,19 +8,13 @@
 package org.opendaylight.controller.md.sal.dom.store.impl;
 
 import static com.google.common.base.Preconditions.checkState;
-
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-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.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-import javax.annotation.concurrent.GuardedBy;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataChangeScope;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.controller.md.sal.common.api.data.OptimisticLockFailedException;
@@ -62,10 +56,10 @@ import org.slf4j.LoggerFactory;
  * to implement {@link DOMStore} contract.
  *
  */
-public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, SchemaContextListener,
-        TransactionReadyPrototype,AutoCloseable {
+public class InMemoryDOMDataStore extends TransactionReadyPrototype implements DOMStore, Identifiable<String>, SchemaContextListener, AutoCloseable {
     private static final Logger LOG = LoggerFactory.getLogger(InMemoryDOMDataStore.class);
     private static final ListenableFuture<Void> SUCCESSFUL_FUTURE = Futures.immediateFuture(null);
+    private static final ListenableFuture<Boolean> CAN_COMMIT_FUTURE = Futures.immediateFuture(Boolean.TRUE);
 
     private static final Invoker<DataChangeListenerRegistration<?>, DOMImmutableDataChangeEvent> DCL_NOTIFICATION_MGR_INVOKER =
             new Invoker<DataChangeListenerRegistration<?>, DOMImmutableDataChangeEvent>() {
@@ -82,29 +76,21 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
     private final DataTree dataTree = InMemoryDataTreeFactory.getInstance().create();
     private final ListenerTree listenerTree = ListenerTree.create();
     private final AtomicLong txCounter = new AtomicLong(0);
-    private final ListeningExecutorService listeningExecutor;
 
     private final QueuedNotificationManager<DataChangeListenerRegistration<?>, DOMImmutableDataChangeEvent> dataChangeListenerNotificationManager;
     private final ExecutorService dataChangeListenerExecutor;
-
-    private final ExecutorService domStoreExecutor;
     private final boolean debugTransactions;
     private final String name;
 
     private volatile AutoCloseable closeable;
 
-    public InMemoryDOMDataStore(final String name, final ExecutorService domStoreExecutor,
-            final ExecutorService dataChangeListenerExecutor) {
-        this(name, domStoreExecutor, dataChangeListenerExecutor,
-             InMemoryDOMDataStoreConfigProperties.DEFAULT_MAX_DATA_CHANGE_LISTENER_QUEUE_SIZE, false);
+    public InMemoryDOMDataStore(final String name, final ExecutorService dataChangeListenerExecutor) {
+        this(name, dataChangeListenerExecutor, InMemoryDOMDataStoreConfigProperties.DEFAULT_MAX_DATA_CHANGE_LISTENER_QUEUE_SIZE, false);
     }
 
-    public InMemoryDOMDataStore(final String name, final ExecutorService domStoreExecutor,
-            final ExecutorService dataChangeListenerExecutor, final int maxDataChangeListenerQueueSize,
-            final boolean debugTransactions) {
+    public InMemoryDOMDataStore(final String name, final ExecutorService dataChangeListenerExecutor,
+            final int maxDataChangeListenerQueueSize, final boolean debugTransactions) {
         this.name = Preconditions.checkNotNull(name);
-        this.domStoreExecutor = Preconditions.checkNotNull(domStoreExecutor);
-        this.listeningExecutor = MoreExecutors.listeningDecorator(this.domStoreExecutor);
         this.dataChangeListenerExecutor = Preconditions.checkNotNull(dataChangeListenerExecutor);
         this.debugTransactions = debugTransactions;
 
@@ -114,7 +100,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
                         "DataChangeListenerQueueMgr");
     }
 
-    public void setCloseable(AutoCloseable closeable) {
+    public void setCloseable(final AutoCloseable closeable) {
         this.closeable = closeable;
     }
 
@@ -122,10 +108,6 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         return dataChangeListenerNotificationManager;
     }
 
-    public ExecutorService getDomStoreExecutor() {
-        return domStoreExecutor;
-    }
-
     @Override
     public final String getIdentifier() {
         return name;
@@ -148,7 +130,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
 
     @Override
     public DOMStoreTransactionChain createTransactionChain() {
-        return new DOMStoreTransactionChainImpl();
+        return new DOMStoreTransactionChainImpl(this);
     }
 
     @Override
@@ -158,7 +140,6 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
 
     @Override
     public void close() {
-        ExecutorServiceUtil.tryGracefulShutdown(listeningExecutor, 30, TimeUnit.SECONDS);
         ExecutorServiceUtil.tryGracefulShutdown(dataChangeListenerExecutor, 30, TimeUnit.SECONDS);
 
         if(closeable != null) {
@@ -170,10 +151,14 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         }
     }
 
-    boolean getDebugTransactions() {
+    public final boolean getDebugTransactions() {
         return debugTransactions;
     }
 
+    final DataTreeSnapshot takeSnapshot() {
+        return dataTree.takeSnapshot();
+    }
+
     @Override
     public <L extends AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>> ListenerRegistration<L> registerChangeListener(
             final YangInstanceIdentifier path, final L listener, final DataChangeScope scope) {
@@ -215,200 +200,69 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
     }
 
     @Override
-    public DOMStoreThreePhaseCommitCohort ready(final SnapshotBackedWriteTransaction writeTx) {
-        LOG.debug("Tx: {} is submitted. Modifications: {}", writeTx.getIdentifier(), writeTx.getMutatedView());
-        return new ThreePhaseCommitImpl(writeTx);
+    protected void transactionAborted(final SnapshotBackedWriteTransaction tx) {
+        LOG.debug("Tx: {} is closed.", tx.getIdentifier());
     }
 
-    private Object nextIdentifier() {
-        return name + "-" + txCounter.getAndIncrement();
-    }
-
-    private class DOMStoreTransactionChainImpl implements DOMStoreTransactionChain, TransactionReadyPrototype {
-
-        @GuardedBy("this")
-        private SnapshotBackedWriteTransaction latestOutstandingTx;
-
-        private boolean chainFailed = false;
-
-        private void checkFailed() {
-            Preconditions.checkState(!chainFailed, "Transaction chain is failed.");
-        }
-
-        @Override
-        public synchronized DOMStoreReadTransaction newReadOnlyTransaction() {
-            final DataTreeSnapshot snapshot;
-            checkFailed();
-            if (latestOutstandingTx != null) {
-                checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
-                snapshot = latestOutstandingTx.getMutatedView();
-            } else {
-                snapshot = dataTree.takeSnapshot();
-            }
-            return new SnapshotBackedReadTransaction(nextIdentifier(), getDebugTransactions(), snapshot);
-        }
-
-        @Override
-        public synchronized DOMStoreReadWriteTransaction newReadWriteTransaction() {
-            final DataTreeSnapshot snapshot;
-            checkFailed();
-            if (latestOutstandingTx != null) {
-                checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
-                snapshot = latestOutstandingTx.getMutatedView();
-            } else {
-                snapshot = dataTree.takeSnapshot();
-            }
-            final SnapshotBackedReadWriteTransaction ret = new SnapshotBackedReadWriteTransaction(nextIdentifier(),
-                    getDebugTransactions(), snapshot, this);
-            latestOutstandingTx = ret;
-            return ret;
-        }
-
-        @Override
-        public synchronized DOMStoreWriteTransaction newWriteOnlyTransaction() {
-            final DataTreeSnapshot snapshot;
-            checkFailed();
-            if (latestOutstandingTx != null) {
-                checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
-                snapshot = latestOutstandingTx.getMutatedView();
-            } else {
-                snapshot = dataTree.takeSnapshot();
-            }
-            final SnapshotBackedWriteTransaction ret = new SnapshotBackedWriteTransaction(nextIdentifier(),
-                    getDebugTransactions(), snapshot, this);
-            latestOutstandingTx = ret;
-            return ret;
-        }
-
-        @Override
-        public DOMStoreThreePhaseCommitCohort ready(final SnapshotBackedWriteTransaction tx) {
-            DOMStoreThreePhaseCommitCohort storeCohort = InMemoryDOMDataStore.this.ready(tx);
-            return new ChainedTransactionCommitImpl(tx, storeCohort, this);
-        }
-
-        @Override
-        public void close() {
-
-            // FIXME: this call doesn't look right here - listeningExecutor is shared and owned
-            // by the outer class.
-            //listeningExecutor.shutdownNow();
-        }
-
-        protected synchronized void onTransactionFailed(final SnapshotBackedWriteTransaction transaction,
-                final Throwable t) {
-            chainFailed = true;
-
-        }
-
-        public synchronized void onTransactionCommited(final SnapshotBackedWriteTransaction transaction) {
-            // If committed transaction is latestOutstandingTx we clear
-            // latestOutstandingTx
-            // field in order to base new transactions on Datastore Data Tree
-            // directly.
-            if (transaction.equals(latestOutstandingTx)) {
-                latestOutstandingTx = null;
-            }
-        }
-
+    @Override
+    protected DOMStoreThreePhaseCommitCohort transactionReady(final SnapshotBackedWriteTransaction tx, final DataTreeModification tree) {
+        LOG.debug("Tx: {} is submitted. Modifications: {}", tx.getIdentifier(), tree);
+        return new ThreePhaseCommitImpl(tx, tree);
     }
 
-    private static class ChainedTransactionCommitImpl implements DOMStoreThreePhaseCommitCohort {
-
-        private final SnapshotBackedWriteTransaction transaction;
-        private final DOMStoreThreePhaseCommitCohort delegate;
-
-        private final DOMStoreTransactionChainImpl txChain;
-
-        protected ChainedTransactionCommitImpl(final SnapshotBackedWriteTransaction transaction,
-                final DOMStoreThreePhaseCommitCohort delegate, final DOMStoreTransactionChainImpl txChain) {
-            super();
-            this.transaction = transaction;
-            this.delegate = delegate;
-            this.txChain = txChain;
-        }
-
-        @Override
-        public ListenableFuture<Boolean> canCommit() {
-            return delegate.canCommit();
-        }
-
-        @Override
-        public ListenableFuture<Void> preCommit() {
-            return delegate.preCommit();
-        }
-
-        @Override
-        public ListenableFuture<Void> abort() {
-            return delegate.abort();
-        }
-
-        @Override
-        public ListenableFuture<Void> commit() {
-            ListenableFuture<Void> commitFuture = delegate.commit();
-            Futures.addCallback(commitFuture, new FutureCallback<Void>() {
-                @Override
-                public void onFailure(final Throwable t) {
-                    txChain.onTransactionFailed(transaction, t);
-                }
-
-                @Override
-                public void onSuccess(final Void result) {
-                    txChain.onTransactionCommited(transaction);
-                }
-
-            });
-            return commitFuture;
-        }
-
+    Object nextIdentifier() {
+        return name + "-" + txCounter.getAndIncrement();
     }
 
-    private class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort {
-
+    private final class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort {
         private final SnapshotBackedWriteTransaction transaction;
         private final DataTreeModification modification;
 
         private ResolveDataChangeEventsTask listenerResolver;
         private DataTreeCandidate candidate;
 
-        public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction writeTransaction) {
+        public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction writeTransaction, final DataTreeModification modification) {
             this.transaction = writeTransaction;
-            this.modification = transaction.getMutatedView();
+            this.modification = modification;
         }
 
         @Override
         public ListenableFuture<Boolean> canCommit() {
-            return listeningExecutor.submit(new Callable<Boolean>() {
-                @Override
-                public Boolean call() throws TransactionCommitFailedException {
-                    try {
-                        dataTree.validate(modification);
-                        LOG.debug("Store Transaction: {} can be committed", transaction.getIdentifier());
-                        return true;
-                    } catch (ConflictingModificationAppliedException e) {
-                        LOG.warn("Store Tx: {} Conflicting modification for {}.", transaction.getIdentifier(),
-                                e.getPath());
-                        transaction.warnDebugContext(LOG);
-                        throw new OptimisticLockFailedException("Optimistic lock failed.",e);
-                    } catch (DataValidationFailedException e) {
-                        LOG.warn("Store Tx: {} Data Precondition failed for {}.", transaction.getIdentifier(),
-                                e.getPath(), e);
-                        transaction.warnDebugContext(LOG);
-                        throw new TransactionCommitFailedException("Data did not pass validation.",e);
-                    }
-                }
-            });
+            try {
+                dataTree.validate(modification);
+                LOG.debug("Store Transaction: {} can be committed", transaction.getIdentifier());
+                return CAN_COMMIT_FUTURE;
+            } catch (ConflictingModificationAppliedException e) {
+                LOG.warn("Store Tx: {} Conflicting modification for {}.", transaction.getIdentifier(),
+                        e.getPath());
+                transaction.warnDebugContext(LOG);
+                return Futures.immediateFailedFuture(new OptimisticLockFailedException("Optimistic lock failed.", e));
+            } catch (DataValidationFailedException e) {
+                LOG.warn("Store Tx: {} Data Precondition failed for {}.", transaction.getIdentifier(),
+                        e.getPath(), e);
+                transaction.warnDebugContext(LOG);
+
+                // For debugging purposes, allow dumping of the modification. Coupled with the above
+                // precondition log, it should allow us to understand what went on.
+                LOG.trace("Store Tx: {} modifications: {} tree: {}", modification, dataTree);
+
+                return Futures.immediateFailedFuture(new TransactionCommitFailedException("Data did not pass validation.", e));
+            } catch (Exception e) {
+                LOG.warn("Unexpected failure in validation phase", e);
+                return Futures.immediateFailedFuture(e);
+            }
         }
 
         @Override
         public ListenableFuture<Void> preCommit() {
-            return listeningExecutor.submit(new Callable<Void>() {
-                @Override
-                public Void call() {
-                    candidate = dataTree.prepare(modification);
-                    listenerResolver = ResolveDataChangeEventsTask.create(candidate, listenerTree);
-                    return null;
-                }
-            });
+            try {
+                candidate = dataTree.prepare(modification);
+                listenerResolver = ResolveDataChangeEventsTask.create(candidate, listenerTree);
+                return SUCCESSFUL_FUTURE;
+            } catch (Exception e) {
+                LOG.warn("Unexpected failure in pre-commit phase", e);
+                return Futures.immediateFailedFuture(e);
+            }
         }
 
         @Override
@@ -425,7 +279,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
              * The commit has to occur atomically with regard to listener
              * registrations.
              */
-            synchronized (this) {
+            synchronized (InMemoryDOMDataStore.this) {
                 dataTree.commit(candidate);
                 listenerResolver.resolve(dataChangeListenerNotificationManager);
             }