Merge "Bug 1073: Introduced Transaction Chain to DOMStore APIs."
[controller.git] / opendaylight / md-sal / sal-dom-broker / src / main / java / org / opendaylight / controller / md / sal / dom / store / impl / InMemoryDOMDataStore.java
index 2091913f24949106aae128e07083a4354b992b70..87c68596efa60a668a7afe005d482eefac1f6ad7 100644 (file)
@@ -9,65 +9,56 @@ package org.opendaylight.controller.md.sal.dom.store.impl;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
-import static org.opendaylight.controller.md.sal.dom.store.impl.StoreUtils.increase;
 
 import java.util.Collections;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
 
 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.dom.store.impl.tree.ListenerRegistrationNode;
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.ListenerRegistrationNode.DataChangeListenerRegistration;
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.ModificationType;
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.NodeModification;
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreMetadataNode;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataPreconditionFailedException;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTree;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeCandidate;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeSnapshot;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.ListenerTree;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.InMemoryDataTreeFactory;
 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.DOMStoreTransaction;
+import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
+import org.opendaylight.yangtools.concepts.AbstractListenerRegistration;
 import org.opendaylight.yangtools.concepts.Identifiable;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
-import org.opendaylight.yangtools.yang.data.impl.schema.NormalizedNodeUtils;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.opendaylight.yangtools.yang.model.api.SchemaContextListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Objects;
+import com.google.common.base.Objects.ToStringHelper;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.primitives.UnsignedLong;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 
 public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, SchemaContextListener {
-
     private static final Logger LOG = LoggerFactory.getLogger(InMemoryDOMDataStore.class);
-    private static final InstanceIdentifier PUBLIC_ROOT_PATH = InstanceIdentifier.builder().build();
-
-
+    private final DataTree dataTree = InMemoryDataTreeFactory.getInstance().create();
+    private final ListenerTree listenerTree = ListenerTree.create();
+    private final AtomicLong txCounter = new AtomicLong(0);
     private final ListeningExecutorService executor;
     private final String name;
-    private final AtomicLong txCounter = new AtomicLong(0);
-    private final ListenerRegistrationNode listenerTree;
-    private final AtomicReference<DataAndMetadataSnapshot> snapshot;
-
-    private ModificationApplyOperation operationTree;
-
-    private SchemaContext schemaContext;
 
     public InMemoryDOMDataStore(final String name, final ListeningExecutorService executor) {
         this.name = Preconditions.checkNotNull(name);
         this.executor = Preconditions.checkNotNull(executor);
-        this.listenerTree = ListenerRegistrationNode.createRoot();
-        this.snapshot = new AtomicReference<DataAndMetadataSnapshot>(DataAndMetadataSnapshot.createEmpty());
-        this.operationTree = new AlwaysFailOperation();
     }
 
     @Override
@@ -77,50 +68,51 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
 
     @Override
     public DOMStoreReadTransaction newReadOnlyTransaction() {
-        return new SnapshotBackedReadTransaction(nextIdentifier(), snapshot.get());
+        return new SnapshotBackedReadTransaction(nextIdentifier(), dataTree.takeSnapshot());
     }
 
     @Override
     public DOMStoreReadWriteTransaction newReadWriteTransaction() {
-        return new SnapshotBackedReadWriteTransaction(nextIdentifier(), snapshot.get(), this, operationTree);
+        return new SnapshotBackedReadWriteTransaction(nextIdentifier(), dataTree.takeSnapshot(), this);
     }
 
     @Override
     public DOMStoreWriteTransaction newWriteOnlyTransaction() {
-        return new SnaphostBackedWriteTransaction(nextIdentifier(), snapshot.get(), this, operationTree);
+        return new SnapshotBackedWriteTransaction(nextIdentifier(), dataTree.takeSnapshot(), this);
+    }
+
+    @Override
+    public DOMStoreTransactionChain createTransactionChain() {
+        throw new UnsupportedOperationException("Not implemented yet.");
     }
 
     @Override
     public synchronized void onGlobalContextUpdated(final SchemaContext ctx) {
-        operationTree = SchemaAwareApplyOperationRoot.from(ctx);
-        schemaContext = ctx;
+        dataTree.setSchemaContext(ctx);
     }
 
     @Override
     public <L extends AsyncDataChangeListener<InstanceIdentifier, NormalizedNode<?, ?>>> ListenerRegistration<L> registerChangeListener(
             final InstanceIdentifier path, final L listener, final DataChangeScope scope) {
-        LOG.debug("{}: Registering data change listener {} for {}",name,listener,path);
-        ListenerRegistrationNode listenerNode = listenerTree;
-        for(PathArgument arg : path.getPath()) {
-            listenerNode = listenerNode.ensureChild(arg);
-        }
 
         /*
-         * Make sure commit is not occurring right now. Listener has to be registered and its
-         * state capture enqueued at a consistent point.
+         * Make sure commit is not occurring right now. Listener has to be
+         * registered and its state capture enqueued at a consistent point.
          *
-         * FIXME: improve this to read-write lock, such that multiple listener registrations
-         *        can occur simultaneously
+         * FIXME: improve this to read-write lock, such that multiple listener
+         * registrations can occur simultaneously
          */
         final DataChangeListenerRegistration<L> reg;
         synchronized (this) {
-            reg = listenerNode.registerDataChangeListener(path, listener, scope);
+            LOG.debug("{}: Registering data change listener {} for {}", name, listener, path);
 
-            Optional<StoreMetadataNode> currentState = snapshot.get().read(path);
+            reg = listenerTree.registerDataChangeListener(path, listener, scope);
+
+            Optional<NormalizedNode<?, ?>> currentState = dataTree.takeSnapshot().readNode(path);
             if (currentState.isPresent()) {
-                final NormalizedNode<?, ?> data = currentState.get().getData();
+                final NormalizedNode<?, ?> data = currentState.get();
 
-                final DOMImmutableDataChangeEvent event = DOMImmutableDataChangeEvent.builder() //
+                final DOMImmutableDataChangeEvent event = DOMImmutableDataChangeEvent.builder(DataChangeScope.BASE) //
                         .setAfter(data) //
                         .addCreated(path, data) //
                         .build();
@@ -128,12 +120,18 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
             }
         }
 
-        return reg;
+        return new AbstractListenerRegistration<L>(listener) {
+            @Override
+            protected void removeRegistration() {
+                synchronized (InMemoryDOMDataStore.this) {
+                    reg.close();
+                }
+            }
+        };
     }
 
-    private synchronized DOMStoreThreePhaseCommitCohort submit(
-            final SnaphostBackedWriteTransaction writeTx) {
-        LOG.debug("Tx: {} is submitted. Modifications: {}",writeTx.getIdentifier(),writeTx.getMutatedView());
+    private synchronized DOMStoreThreePhaseCommitCohort submit(final SnapshotBackedWriteTransaction writeTx) {
+        LOG.debug("Tx: {} is submitted. Modifications: {}", writeTx.getIdentifier(), writeTx.getMutatedView());
         return new ThreePhaseCommitImpl(writeTx);
     }
 
@@ -141,51 +139,48 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         return name + "-" + txCounter.getAndIncrement();
     }
 
-    private void commit(final DataAndMetadataSnapshot currentSnapshot,
-            final StoreMetadataNode newDataTree, final Iterable<ChangeListenerNotifyTask> listenerTasks) {
-        LOG.debug("Updating Store snaphot version: {} with version:{}",currentSnapshot.getMetadataTree().getSubtreeVersion(),newDataTree.getSubtreeVersion());
+    private static abstract class AbstractDOMStoreTransaction implements DOMStoreTransaction {
+        private final Object identifier;
 
-        if(LOG.isTraceEnabled()) {
-            LOG.trace("Data Tree is {}",StoreUtils.toStringTree(newDataTree));
+        protected AbstractDOMStoreTransaction(final Object identifier) {
+            this.identifier = identifier;
         }
 
-        final DataAndMetadataSnapshot newSnapshot = DataAndMetadataSnapshot.builder() //
-                .setMetadataTree(newDataTree) //
-                .setSchemaContext(schemaContext) //
-                .build();
+        @Override
+        public final Object getIdentifier() {
+            return identifier;
+        }
 
-        /*
-         * The commit has to occur atomically with regard to listener registrations.
-         */
-        synchronized (this) {
-            final boolean success = snapshot.compareAndSet(currentSnapshot, newSnapshot);
-            checkState(success, "Store snapshot and transaction snapshot differ. This should never happen.");
+        @Override
+        public final String toString() {
+            return addToStringAttributes(Objects.toStringHelper(this)).toString();
+        }
 
-            for (ChangeListenerNotifyTask task : listenerTasks) {
-                executor.submit(task);
-            }
+        /**
+         * Add class-specific toString attributes.
+         *
+         * @param toStringHelper
+         *            ToStringHelper instance
+         * @return ToStringHelper instance which was passed in
+         */
+        protected ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
+            return toStringHelper.add("id", identifier);
         }
     }
 
-    private static class SnapshotBackedReadTransaction implements DOMStoreReadTransaction {
-
-        private DataAndMetadataSnapshot stableSnapshot;
-        private final Object identifier;
-
-        public SnapshotBackedReadTransaction(final Object identifier, final DataAndMetadataSnapshot snapshot) {
-            this.identifier = identifier;
-            this.stableSnapshot = snapshot;
-            LOG.debug("ReadOnly Tx: {} allocated with snapshot {}",identifier,snapshot.getMetadataTree().getSubtreeVersion());
-
-        }
+    private static final class SnapshotBackedReadTransaction extends AbstractDOMStoreTransaction implements
+    DOMStoreReadTransaction {
+        private DataTreeSnapshot stableSnapshot;
 
-        @Override
-        public Object getIdentifier() {
-            return identifier;
+        public SnapshotBackedReadTransaction(final Object identifier, final DataTreeSnapshot snapshot) {
+            super(identifier);
+            this.stableSnapshot = Preconditions.checkNotNull(snapshot);
+            LOG.debug("ReadOnly Tx: {} allocated with snapshot {}", identifier, snapshot);
         }
 
         @Override
         public void close() {
+            LOG.debug("Store transaction: {} : Closed", getIdentifier());
             stableSnapshot = null;
         }
 
@@ -193,39 +188,27 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         public ListenableFuture<Optional<NormalizedNode<?, ?>>> read(final InstanceIdentifier path) {
             checkNotNull(path, "Path must not be null.");
             checkState(stableSnapshot != null, "Transaction is closed");
-            return Futures.immediateFuture(NormalizedNodeUtils.findNode(stableSnapshot.getDataTree(), path));
-        }
-
-        @Override
-        public String toString() {
-            return "SnapshotBackedReadTransaction [id =" + identifier + "]";
+            return Futures.immediateFuture(stableSnapshot.readNode(path));
         }
-
     }
 
-    private static class SnaphostBackedWriteTransaction implements DOMStoreWriteTransaction {
-
-        private MutableDataTree mutableTree;
-        private final Object identifier;
+    private static class SnapshotBackedWriteTransaction extends AbstractDOMStoreTransaction implements
+    DOMStoreWriteTransaction {
+        private DataTreeModification mutableTree;
         private InMemoryDOMDataStore store;
-
         private boolean ready = false;
 
-        public SnaphostBackedWriteTransaction(final Object identifier, final DataAndMetadataSnapshot snapshot,
-                final InMemoryDOMDataStore store, final ModificationApplyOperation applyOper) {
-            this.identifier = identifier;
-            mutableTree = MutableDataTree.from(snapshot, applyOper);
+        public SnapshotBackedWriteTransaction(final Object identifier, final DataTreeSnapshot snapshot,
+                final InMemoryDOMDataStore store) {
+            super(identifier);
+            mutableTree = snapshot.newModification();
             this.store = store;
-            LOG.debug("Write Tx: {} allocated with snapshot {}",identifier,snapshot.getMetadataTree().getSubtreeVersion());
-        }
-
-        @Override
-        public Object getIdentifier() {
-            return identifier;
+            LOG.debug("Write Tx: {} allocated with snapshot {}", identifier, snapshot);
         }
 
         @Override
         public void close() {
+            LOG.debug("Store transaction: {} : Closed", getIdentifier());
             this.mutableTree = null;
             this.store = null;
         }
@@ -233,118 +216,124 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         @Override
         public void write(final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
             checkNotReady();
-            mutableTree.write(path, data);
+            try {
+                LOG.trace("Tx: {} Write: {}:{}", getIdentifier(), path, data);
+                mutableTree.write(path, data);
+                // FIXME: Add checked exception
+            } catch (Exception e) {
+                LOG.error("Tx: {}, failed to write {}:{} in {}", getIdentifier(), path, data, mutableTree, e);
+            }
+        }
+
+        @Override
+        public void merge(final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
+            checkNotReady();
+            try {
+                LOG.trace("Tx: {} Merge: {}:{}", getIdentifier(), path, data);
+                mutableTree.merge(path, data);
+                // FIXME: Add checked exception
+            } catch (Exception e) {
+                LOG.error("Tx: {}, failed to write {}:{} in {}", getIdentifier(), path, data, mutableTree, e);
+            }
         }
 
         @Override
         public void delete(final InstanceIdentifier path) {
             checkNotReady();
-            mutableTree.delete(path);
+            try {
+                LOG.trace("Tx: {} Delete: {}", getIdentifier(), path);
+                mutableTree.delete(path);
+                // FIXME: Add checked exception
+            } catch (Exception e) {
+                LOG.error("Tx: {}, failed to delete {} in {}", getIdentifier(), path, mutableTree, e);
+            }
         }
 
-        protected boolean isReady() {
+        protected final boolean isReady() {
             return ready;
         }
 
-        protected void checkNotReady() {
-            checkState(!ready, "Transaction is ready. No further modifications allowed.");
+        protected final void checkNotReady() {
+            checkState(!ready, "Transaction %s is ready. No further modifications allowed.", getIdentifier());
         }
 
         @Override
         public synchronized DOMStoreThreePhaseCommitCohort ready() {
+            checkState(!ready, "Transaction %s is already ready.", getIdentifier());
             ready = true;
+
             LOG.debug("Store transaction: {} : Ready", getIdentifier());
-            mutableTree.seal();
+            mutableTree.ready();
             return store.submit(this);
         }
 
-        protected MutableDataTree getMutatedView() {
+        protected DataTreeModification getMutatedView() {
             return mutableTree;
         }
 
         @Override
-        public String toString() {
-            return "SnaphostBackedWriteTransaction [id=" + getIdentifier() + ", ready=" + isReady() + "]";
+        protected ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
+            return toStringHelper.add("ready", isReady());
         }
-
     }
 
-    private static class SnapshotBackedReadWriteTransaction extends SnaphostBackedWriteTransaction implements
-            DOMStoreReadWriteTransaction {
+    private static class SnapshotBackedReadWriteTransaction extends SnapshotBackedWriteTransaction implements
+    DOMStoreReadWriteTransaction {
 
-        protected SnapshotBackedReadWriteTransaction(final Object identifier, final DataAndMetadataSnapshot snapshot,
-                final InMemoryDOMDataStore store, final ModificationApplyOperation applyOper) {
-            super(identifier, snapshot, store, applyOper);
+        protected SnapshotBackedReadWriteTransaction(final Object identifier, final DataTreeSnapshot snapshot,
+                final InMemoryDOMDataStore store) {
+            super(identifier, snapshot, store);
         }
 
         @Override
         public ListenableFuture<Optional<NormalizedNode<?, ?>>> read(final InstanceIdentifier path) {
-            return Futures.immediateFuture(getMutatedView().read(path));
-        }
-
-        @Override
-        public String toString() {
-            return "SnapshotBackedReadWriteTransaction [id=" + getIdentifier() + ", ready=" + isReady() + "]";
+            LOG.trace("Tx: {} Read: {}", getIdentifier(), path);
+            try {
+                return Futures.immediateFuture(getMutatedView().readNode(path));
+            } catch (Exception e) {
+                LOG.error("Tx: {} Failed Read of {}", getIdentifier(), path, e);
+                throw e;
+            }
         }
-
     }
 
     private class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort {
 
-        private final SnaphostBackedWriteTransaction transaction;
-        private final NodeModification modification;
+        private final SnapshotBackedWriteTransaction transaction;
+        private final DataTreeModification modification;
 
-        private DataAndMetadataSnapshot storeSnapshot;
-        private Optional<StoreMetadataNode> proposedSubtree;
-        private Iterable<ChangeListenerNotifyTask> listenerTasks;
+        private ResolveDataChangeEventsTask listenerResolver;
+        private DataTreeCandidate candidate;
 
-        public ThreePhaseCommitImpl(final SnaphostBackedWriteTransaction writeTransaction) {
+        public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction writeTransaction) {
             this.transaction = writeTransaction;
-            this.modification = transaction.getMutatedView().getRootModification();
+            this.modification = transaction.getMutatedView();
         }
 
         @Override
         public ListenableFuture<Boolean> canCommit() {
-            final DataAndMetadataSnapshot snapshotCapture = snapshot.get();
-            final ModificationApplyOperation snapshotOperation = operationTree;
-
             return executor.submit(new Callable<Boolean>() {
-
                 @Override
-                public Boolean call() throws Exception {
-                    boolean applicable = snapshotOperation.isApplicable(modification,
-                            Optional.of(snapshotCapture.getMetadataTree()));
-                    LOG.debug("Store Transcation: {} : canCommit : {}", transaction.getIdentifier(), applicable);
-                    return applicable;
+                public Boolean call() {
+                    try {
+                        dataTree.validate(modification);
+                        LOG.debug("Store Transaction: {} can be committed", transaction.getIdentifier());
+                        return true;
+                    } catch (DataPreconditionFailedException e) {
+                        LOG.warn("Store Tx: {} Data Precondition failed for {}.",transaction.getIdentifier(),e.getPath(),e);
+                        return false;
+                    }
                 }
             });
         }
 
         @Override
         public ListenableFuture<Void> preCommit() {
-            storeSnapshot = snapshot.get();
-            if(modification.getModificationType() == ModificationType.UNMODIFIED) {
-                return Futures.immediateFuture(null);
-            }
             return executor.submit(new Callable<Void>() {
-
-
-
                 @Override
-                public Void call() throws Exception {
-                    StoreMetadataNode metadataTree = storeSnapshot.getMetadataTree();
-
-                    proposedSubtree = operationTree.apply(modification, Optional.of(metadataTree),
-                            increase(metadataTree.getSubtreeVersion()));
-
-                    listenerTasks = DataChangeEventResolver.create() //
-                            .setRootPath(PUBLIC_ROOT_PATH) //
-                            .setBeforeRoot(Optional.of(metadataTree)) //
-                            .setAfterRoot(proposedSubtree) //
-                            .setModificationRoot(modification) //
-                            .setListenerRoot(listenerTree) //
-                            .resolve();
-
+                public Void call() {
+                    candidate = dataTree.prepare(modification);
+                    listenerResolver = ResolveDataChangeEventsTask.create(candidate, listenerTree);
                     return null;
                 }
             });
@@ -352,48 +341,28 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
 
         @Override
         public ListenableFuture<Void> abort() {
-            storeSnapshot = null;
-            proposedSubtree = null;
-            return Futures.<Void> immediateFuture(null);
+            candidate = null;
+            return Futures.immediateFuture(null);
         }
 
         @Override
         public ListenableFuture<Void> commit() {
-            if(modification.getModificationType() == ModificationType.UNMODIFIED) {
-                return Futures.immediateFuture(null);
+            checkState(candidate != null, "Proposed subtree must be computed");
+
+            /*
+             * The commit has to occur atomically with regard to listener
+             * registrations.
+             */
+            synchronized (this) {
+                dataTree.commit(candidate);
+
+                for (ChangeListenerNotifyTask task : listenerResolver.call()) {
+                    LOG.trace("Scheduling invocation of listeners: {}", task);
+                    executor.submit(task);
+                }
             }
 
-            checkState(proposedSubtree != null,"Proposed subtree must be computed");
-            checkState(storeSnapshot != null,"Proposed subtree must be computed");
-            // return ImmediateFuture<>;
-            InMemoryDOMDataStore.this.commit(storeSnapshot, proposedSubtree.get(),listenerTasks);
-            return Futures.<Void> immediateFuture(null);
-        }
-
-    }
-
-    private static final class AlwaysFailOperation implements ModificationApplyOperation {
-
-        @Override
-        public Optional<StoreMetadataNode> apply(final NodeModification modification,
-                final Optional<StoreMetadataNode> storeMeta, final UnsignedLong subtreeVersion) {
-            throw new IllegalStateException("Schema Context is not available.");
+            return Futures.immediateFuture(null);
         }
-
-        @Override
-        public boolean isApplicable(final NodeModification modification, final Optional<StoreMetadataNode> storeMetadata) {
-            throw new IllegalStateException("Schema Context is not available.");
-        }
-
-        @Override
-        public Optional<ModificationApplyOperation> getChild(final PathArgument child) {
-            throw new IllegalStateException("Schema Context is not available.");
-        }
-
-        @Override
-        public void verifyStructure(final NodeModification modification) throws IllegalArgumentException {
-            throw new IllegalStateException("Schema Context is not available.");
-        }
-
     }
 }