Delay snapshot backed transaction ready error
[controller.git] / opendaylight / md-sal / sal-inmemory-datastore / src / main / java / org / opendaylight / controller / md / sal / dom / store / impl / InMemoryDOMDataStore.java
index 354abcf69fe1040a1e24822b7db78aa6e674909d..3582ff6ef1061749f1649cc58aed21def801d002 100644 (file)
@@ -7,22 +7,16 @@
  */
 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.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
+import java.util.Optional;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 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;
-import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
+import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeChangeListener;
-import org.opendaylight.controller.md.sal.dom.store.impl.SnapshotBackedWriteTransaction.TransactionReadyPrototype;
 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;
@@ -30,6 +24,9 @@ import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCoh
 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;
@@ -38,9 +35,9 @@ import org.opendaylight.yangtools.util.concurrent.QueuedNotificationManager;
 import org.opendaylight.yangtools.util.concurrent.QueuedNotificationManager.Invoker;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.ConflictingModificationAppliedException;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeConfiguration;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeSnapshot;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
@@ -55,28 +52,22 @@ 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, DOMStoreTreeChangePublisher {
+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>() {
-                @Override
-                public void invokeListener(final DataChangeListenerRegistration<?> listener,
-                                           final DOMImmutableDataChangeEvent notification ) {
-                    final AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> inst = listener.getInstance();
-                    if (inst != null) {
-                        inst.onDataChanged(notification);
-                    }
+            (listener, notification) -> {
+                final AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> inst = listener.getInstance();
+                if (inst != null) {
+                    inst.onDataChanged(notification);
                 }
             };
 
-    private final DataTree dataTree = InMemoryDataTreeFactory.getInstance().create();
+    private final DataTree dataTree;
     private final ListenerTree listenerTree = ListenerTree.create();
     private final AtomicLong txCounter = new AtomicLong(0);
 
@@ -89,10 +80,12 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
     private volatile AutoCloseable closeable;
 
     public InMemoryDOMDataStore(final String name, final ExecutorService dataChangeListenerExecutor) {
-        this(name, dataChangeListenerExecutor, InMemoryDOMDataStoreConfigProperties.DEFAULT_MAX_DATA_CHANGE_LISTENER_QUEUE_SIZE, false);
+        this(name, LogicalDatastoreType.OPERATIONAL, dataChangeListenerExecutor,
+            InMemoryDOMDataStoreConfigProperties.DEFAULT_MAX_DATA_CHANGE_LISTENER_QUEUE_SIZE, false);
     }
 
-    public InMemoryDOMDataStore(final String name, final ExecutorService dataChangeListenerExecutor,
+    public InMemoryDOMDataStore(final String name, final LogicalDatastoreType type,
+            final ExecutorService dataChangeListenerExecutor,
             final int maxDataChangeListenerQueueSize, final boolean debugTransactions) {
         this.name = Preconditions.checkNotNull(name);
         this.dataChangeListenerExecutor = Preconditions.checkNotNull(dataChangeListenerExecutor);
@@ -103,6 +96,17 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
                         DCL_NOTIFICATION_MGR_INVOKER, maxDataChangeListenerQueueSize,
                         "DataChangeListenerQueueMgr");
         changePublisher = new InMemoryDOMStoreTreeChangePublisher(this.dataChangeListenerExecutor, maxDataChangeListenerQueueSize);
+
+        switch (type) {
+            case CONFIGURATION:
+                dataTree = new InMemoryDataTreeFactory().create(DataTreeConfiguration.DEFAULT_CONFIGURATION);
+                break;
+            case OPERATIONAL:
+                dataTree = new InMemoryDataTreeFactory().create(DataTreeConfiguration.DEFAULT_OPERATIONAL);
+                break;
+            default:
+                throw new IllegalArgumentException("Data store " + type + " not supported");
+        }
     }
 
     public void setCloseable(final AutoCloseable closeable) {
@@ -120,17 +124,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
@@ -214,99 +218,33 @@ public class InMemoryDOMDataStore extends TransactionReadyPrototype implements D
     }
 
     @Override
-    protected void transactionAborted(final SnapshotBackedWriteTransaction tx) {
+    protected void transactionAborted(final SnapshotBackedWriteTransaction<String> tx) {
         LOG.debug("Tx: {} is closed.", tx.getIdentifier());
     }
 
     @Override
-    protected DOMStoreThreePhaseCommitCohort transactionReady(final SnapshotBackedWriteTransaction tx, final DataTreeModification tree) {
-        LOG.debug("Tx: {} is submitted. Modifications: {}", tx.getIdentifier(), tree);
-        return new ThreePhaseCommitImpl(tx, tree);
+    protected DOMStoreThreePhaseCommitCohort transactionReady(final SnapshotBackedWriteTransaction<String> tx,
+                                                              final DataTreeModification modification,
+                                                              final Exception readyError) {
+        LOG.debug("Tx: {} is submitted. Modifications: {}", tx.getIdentifier(), modification);
+        return new InMemoryDOMStoreThreePhaseCommitCohort(this, tx, modification, readyError);
     }
 
-    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);
-        }
+    void validate(final DataTreeModification modification) throws DataValidationFailedException {
+        dataTree.validate(modification);
     }
 
-    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, final DataTreeModification modification) {
-            this.transaction = writeTransaction;
-            this.modification = modification;
-        }
-
-        @Override
-        public ListenableFuture<Boolean> canCommit() {
-            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() {
-            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
-        public ListenableFuture<Void> abort() {
-            candidate = null;
-            return SUCCESSFUL_FUTURE;
-        }
-
-        @Override
-        public ListenableFuture<Void> commit() {
-            checkState(candidate != null, "Proposed subtree must be computed");
-
-            /*
-             * The commit has to occur atomically with regard to listener
-             * registrations.
-             */
-            synchronized (InMemoryDOMDataStore.this) {
-                dataTree.commit(candidate);
-                changePublisher.publishChange(candidate);
-                listenerResolver.resolve(dataChangeListenerNotificationManager);
-            }
+    DataTreeCandidate prepare(final DataTreeModification modification) {
+        return dataTree.prepare(modification);
+    }
 
-            return SUCCESSFUL_FUTURE;
-        }
+    synchronized void commit(final DataTreeCandidate candidate) {
+        dataTree.commit(candidate);
+        changePublisher.publishChange(candidate);
+        ResolveDataChangeEventsTask.create(candidate, listenerTree).resolve(dataChangeListenerNotificationManager);
     }
 }