Publish SnapshotBacked transactions
[controller.git] / opendaylight / md-sal / sal-inmemory-datastore / src / main / java / org / opendaylight / controller / md / sal / dom / store / impl / InMemoryDOMDataStore.java
index 213f60e951cc41794864fcb1fd813a1958469728..7e4056a1900b99a777096910b32e6b37d0012574 100644 (file)
@@ -12,8 +12,6 @@ import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -21,14 +19,19 @@ import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataCh
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.controller.md.sal.common.api.data.OptimisticLockFailedException;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
-import org.opendaylight.controller.md.sal.dom.store.impl.SnapshotBackedWriteTransaction.TransactionReadyPrototype;
+import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeChangeListener;
 import org.opendaylight.controller.md.sal.dom.store.impl.tree.ListenerTree;
+import org.opendaylight.controller.sal.core.spi.data.AbstractDOMStoreTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStore;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
+import org.opendaylight.controller.sal.core.spi.data.DOMStoreTreeChangePublisher;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
+import org.opendaylight.controller.sal.core.spi.data.SnapshotBackedTransactions;
+import org.opendaylight.controller.sal.core.spi.data.SnapshotBackedWriteTransaction;
+import org.opendaylight.controller.sal.core.spi.data.SnapshotBackedWriteTransaction.TransactionReadyPrototype;
 import org.opendaylight.yangtools.concepts.AbstractListenerRegistration;
 import org.opendaylight.yangtools.concepts.Identifiable;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
@@ -54,13 +57,14 @@ import org.slf4j.LoggerFactory;
  *
  * Implementation of {@link DOMStore} which uses {@link DataTree} and other
  * classes such as {@link SnapshotBackedWriteTransaction}.
- * {@link SnapshotBackedReadTransaction} and {@link ResolveDataChangeEventsTask}
+ * {@link org.opendaylight.controller.sal.core.spi.data.SnapshotBackedReadTransaction} and {@link ResolveDataChangeEventsTask}
  * to implement {@link DOMStore} contract.
  *
  */
-public class InMemoryDOMDataStore extends TransactionReadyPrototype implements DOMStore, Identifiable<String>, SchemaContextListener, AutoCloseable {
+public class InMemoryDOMDataStore extends TransactionReadyPrototype<String> implements DOMStore, Identifiable<String>, SchemaContextListener, AutoCloseable, DOMStoreTreeChangePublisher {
     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>() {
@@ -79,24 +83,20 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
     private final AtomicLong txCounter = new AtomicLong(0);
 
     private final QueuedNotificationManager<DataChangeListenerRegistration<?>, DOMImmutableDataChangeEvent> dataChangeListenerNotificationManager;
+    private final InMemoryDOMStoreTreeChangePublisher changePublisher;
     private final ExecutorService dataChangeListenerExecutor;
-    private final ListeningExecutorService commitExecutor;
     private final boolean debugTransactions;
     private final String name;
 
     private volatile AutoCloseable closeable;
 
-    public InMemoryDOMDataStore(final String name, final ListeningExecutorService commitExecutor,
-            final ExecutorService dataChangeListenerExecutor) {
-        this(name, commitExecutor, 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 ListeningExecutorService commitExecutor,
-            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.commitExecutor = Preconditions.checkNotNull(commitExecutor);
         this.dataChangeListenerExecutor = Preconditions.checkNotNull(dataChangeListenerExecutor);
         this.debugTransactions = debugTransactions;
 
@@ -104,6 +104,7 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
                 new QueuedNotificationManager<>(this.dataChangeListenerExecutor,
                         DCL_NOTIFICATION_MGR_INVOKER, maxDataChangeListenerQueueSize,
                         "DataChangeListenerQueueMgr");
+        changePublisher = new InMemoryDOMStoreTreeChangePublisher(this.dataChangeListenerExecutor, maxDataChangeListenerQueueSize);
     }
 
     public void setCloseable(final AutoCloseable closeable) {
@@ -114,10 +115,6 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
         return dataChangeListenerNotificationManager;
     }
 
-    public ExecutorService getDomStoreExecutor() {
-        return commitExecutor;
-    }
-
     @Override
     public final String getIdentifier() {
         return name;
@@ -125,17 +122,17 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
 
     @Override
     public DOMStoreReadTransaction newReadOnlyTransaction() {
-        return new SnapshotBackedReadTransaction(nextIdentifier(), debugTransactions, dataTree.takeSnapshot());
+        return SnapshotBackedTransactions.newReadTransaction(nextIdentifier(), debugTransactions, dataTree.takeSnapshot());
     }
 
     @Override
     public DOMStoreReadWriteTransaction newReadWriteTransaction() {
-        return new SnapshotBackedReadWriteTransaction(nextIdentifier(), debugTransactions, dataTree.takeSnapshot(), this);
+        return SnapshotBackedTransactions.newReadWriteTransaction(nextIdentifier(), debugTransactions, dataTree.takeSnapshot(), this);
     }
 
     @Override
     public DOMStoreWriteTransaction newWriteOnlyTransaction() {
-        return new SnapshotBackedWriteTransaction(nextIdentifier(), debugTransactions, dataTree.takeSnapshot(), this);
+        return SnapshotBackedTransactions.newWriteTransaction(nextIdentifier(), debugTransactions, dataTree.takeSnapshot(), this);
     }
 
     @Override
@@ -150,7 +147,6 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
 
     @Override
     public void close() {
-        ExecutorServiceUtil.tryGracefulShutdown(commitExecutor, 30, TimeUnit.SECONDS);
         ExecutorServiceUtil.tryGracefulShutdown(dataChangeListenerExecutor, 30, TimeUnit.SECONDS);
 
         if(closeable != null) {
@@ -211,66 +207,85 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
     }
 
     @Override
-    protected void transactionAborted(final SnapshotBackedWriteTransaction tx) {
+    public synchronized <L extends DOMDataTreeChangeListener> ListenerRegistration<L> registerTreeChangeListener(final YangInstanceIdentifier treeId, final L listener) {
+        /*
+         * Make sure commit is not occurring right now. Listener has to be
+         * registered and its state capture enqueued at a consistent point.
+         */
+        return changePublisher.registerTreeChangeListener(treeId, listener, dataTree.takeSnapshot());
+    }
+
+    @Override
+    protected void transactionAborted(final SnapshotBackedWriteTransaction<String> tx) {
         LOG.debug("Tx: {} is closed.", tx.getIdentifier());
     }
 
     @Override
-    protected DOMStoreThreePhaseCommitCohort transactionReady(final SnapshotBackedWriteTransaction tx, final DataTreeModification tree) {
+    protected DOMStoreThreePhaseCommitCohort transactionReady(final SnapshotBackedWriteTransaction<String> tx, final DataTreeModification tree) {
         LOG.debug("Tx: {} is submitted. Modifications: {}", tx.getIdentifier(), tree);
         return new ThreePhaseCommitImpl(tx, tree);
     }
 
-    Object nextIdentifier() {
+    String nextIdentifier() {
         return name + "-" + txCounter.getAndIncrement();
     }
 
+    private static void warnDebugContext(final AbstractDOMStoreTransaction<?> transaction) {
+        final Throwable ctx = transaction.getDebugContext();
+        if (ctx != null) {
+            LOG.warn("Transaction {} has been allocated in the following context", transaction.getIdentifier(), ctx);
+        }
+    }
+
     private final class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort {
-        private final SnapshotBackedWriteTransaction transaction;
+        private final SnapshotBackedWriteTransaction<String> transaction;
         private final DataTreeModification modification;
 
         private ResolveDataChangeEventsTask listenerResolver;
         private DataTreeCandidate candidate;
 
-        public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction writeTransaction, final DataTreeModification modification) {
+        public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction<String> writeTransaction, final DataTreeModification modification) {
             this.transaction = writeTransaction;
             this.modification = modification;
         }
 
         @Override
         public ListenableFuture<Boolean> canCommit() {
-            return commitExecutor.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());
+                warnDebugContext(transaction);
+                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);
+                warnDebugContext(transaction);
+
+                // 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 commitExecutor.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
@@ -289,6 +304,7 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
              */
             synchronized (InMemoryDOMDataStore.this) {
                 dataTree.commit(candidate);
+                changePublisher.publishChange(candidate);
                 listenerResolver.resolve(dataChangeListenerNotificationManager);
             }