BUG-509: Move DataTree concepts into separate package 23/7223/2
authorRobert Varga <rovarga@cisco.com>
Mon, 19 May 2014 09:56:10 +0000 (11:56 +0200)
committerRobert Varga <rovarga@cisco.com>
Tue, 20 May 2014 12:08:57 +0000 (14:08 +0200)
Create a new package for the data tree and related concepts to clean out
interactions. Also create and document the API and hide implementation
classes from the user itself.

Change-Id: I69f2a9b7bcac863b531f46f669bfd2fcfcd5b743
Signed-off-by: Robert Varga <rovarga@cisco.com>
26 files changed:
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataPreconditionFailedException.java [deleted file]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataTree.java [deleted file]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/OperationWithModification.java
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ResolveDataChangeEventsTask.java
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperation.java
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataPreconditionFailedException.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTree.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeCandidate.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeFactory.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeModification.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeSnapshot.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/ModificationApplyOperation.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationApplyOperation.java with 92% similarity]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreUtils.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/StoreUtils.java with 98% similarity]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/AbstractDataTreeCandidate.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTree.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeCandidate.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeFactory.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeModification.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataTreeModification.java with 75% similarity]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeSnapshot.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationType.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/ModificationType.java with 91% similarity]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/NodeModification.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/NodeModification.java with 97% similarity]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/NoopDataTreeCandidate.java [new file with mode: 0644]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/StoreMetadataNode.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreMetadataNode.java with 97% similarity]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/StoreNodeCompositeBuilder.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreNodeCompositeBuilder.java with 97% similarity]
opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationMetadataTreeTest.java [moved from opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationMetadataTreeTest.java with 86% similarity]

diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataPreconditionFailedException.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataPreconditionFailedException.java
deleted file mode 100644 (file)
index 6baf764..0000000
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-package org.opendaylight.controller.md.sal.dom.store.impl;
-
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
-
-public class DataPreconditionFailedException extends Exception {
-
-    /**
-     *
-     */
-    private static final long serialVersionUID = 596430355175413427L;
-    private final InstanceIdentifier path;
-
-    public DataPreconditionFailedException(final InstanceIdentifier path) {
-        this.path = path;
-    }
-
-    public DataPreconditionFailedException(final InstanceIdentifier path,final String message) {
-        super(message);
-        this.path = path;
-    }
-
-
-    public DataPreconditionFailedException(final InstanceIdentifier path,final Throwable cause) {
-        super(cause);
-        this.path = path;
-    }
-
-    public DataPreconditionFailedException(final InstanceIdentifier path,final String message, final Throwable cause) {
-        super(message, cause);
-        this.path = path;
-    }
-
-    public DataPreconditionFailedException(final InstanceIdentifier path,final String message, final Throwable cause, final boolean enableSuppression,
-            final boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-        this.path = path;
-    }
-
-    public InstanceIdentifier getPath() {
-        return path;
-    }
-
-}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataTree.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataTree.java
deleted file mode 100644 (file)
index 3124199..0000000
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-package org.opendaylight.controller.md.sal.dom.store.impl;
-
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreMetadataNode;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.NodeIdentifier;
-import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
-import org.opendaylight.yangtools.yang.data.impl.schema.Builders;
-import org.opendaylight.yangtools.yang.data.impl.schema.NormalizedNodeUtils;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-
-/**
- * Read-only snapshot of the data tree.
- */
-final class DataTree {
-    public static final class Snapshot {
-        private final SchemaContext schemaContext;
-        private final StoreMetadataNode rootNode;
-
-        @VisibleForTesting
-        Snapshot(final SchemaContext schemaContext, final StoreMetadataNode rootNode) {
-            this.schemaContext = Preconditions.checkNotNull(schemaContext);
-            this.rootNode = Preconditions.checkNotNull(rootNode);
-        }
-
-        public SchemaContext getSchemaContext() {
-            return schemaContext;
-        }
-
-        public Optional<NormalizedNode<?, ?>> readNode(final InstanceIdentifier path) {
-            return NormalizedNodeUtils.findNode(rootNode.getData(), path);
-        }
-
-        // FIXME: this is a leak of information
-        @Deprecated
-        StoreMetadataNode getRootNode() {
-            return rootNode;
-        }
-
-        @Override
-        public String toString() {
-            return rootNode.getSubtreeVersion().toString();
-        }
-    }
-
-    private static final Logger LOG = LoggerFactory.getLogger(DataTree.class);
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock(true);
-    private StoreMetadataNode rootNode;
-    private SchemaContext currentSchemaContext;
-
-    private DataTree(StoreMetadataNode rootNode, final SchemaContext schemaContext) {
-        this.rootNode = Preconditions.checkNotNull(rootNode);
-        this.currentSchemaContext = schemaContext;
-    }
-
-    public synchronized void setSchemaContext(final SchemaContext newSchemaContext) {
-        Preconditions.checkNotNull(newSchemaContext);
-
-        LOG.info("Attepting to install schema context {}", newSchemaContext);
-
-        /*
-         * FIXME: we should walk the schema contexts, both current and new and see
-         *        whether they are compatible here. Reject incompatible changes.
-         */
-
-        // Ready to change the context now, make sure no operations are running
-        rwLock.writeLock().lock();
-        try {
-            this.currentSchemaContext = newSchemaContext;
-        } finally {
-            rwLock.writeLock().unlock();
-        }
-    }
-
-    public static DataTree create(final SchemaContext schemaContext) {
-        final NodeIdentifier root = new NodeIdentifier(SchemaContext.NAME);
-        final NormalizedNode<?, ?> data = Builders.containerBuilder().withNodeIdentifier(root).build();
-
-        return new DataTree(StoreMetadataNode.createEmpty(data), schemaContext);
-    }
-
-    public Snapshot takeSnapshot() {
-        rwLock.readLock().lock();
-
-        try {
-            return new Snapshot(currentSchemaContext, rootNode);
-        } finally {
-            rwLock.readLock().unlock();
-        }
-    }
-
-    public void commitSnapshot(Snapshot currentSnapshot, StoreMetadataNode newDataTree) {
-        // Ready to change the context now, make sure no operations are running
-        rwLock.writeLock().lock();
-        try {
-            Preconditions.checkState(currentSnapshot.rootNode == rootNode,
-                    String.format("Store snapshot %s and transaction snapshot %s differ.",
-                            rootNode, currentSnapshot.rootNode));
-
-            this.rootNode = newDataTree;
-        } finally {
-            rwLock.writeLock().unlock();
-        }
-    }
-}
index 427e7a00dbb443e1bc0f5de8553aa66ab5fc63c8..7d647af53907242e9c06dec14dcfa81e05c2fad6 100644 (file)
@@ -9,7 +9,6 @@ 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;
@@ -17,10 +16,16 @@ 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.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.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.ModificationApplyOperation;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.InMemoryDataTreeFactory;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.NodeModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.StoreMetadataNode;
 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;
@@ -56,7 +61,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
     private final String name;
     private final AtomicLong txCounter = new AtomicLong(0);
     private final ListenerTree listenerTree = ListenerTree.create();
-    private final DataTree dataTree = DataTree.create(null);
+    private final DataTree dataTree = InMemoryDataTreeFactory.getInstance().create();
     private ModificationApplyOperation operationTree = new AlwaysFailOperation();
 
     public InMemoryDOMDataStore(final String name, final ListeningExecutorService executor) {
@@ -145,28 +150,6 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         return name + "-" + txCounter.getAndIncrement();
     }
 
-    private void commit(final DataTree.Snapshot currentSnapshot, final StoreMetadataNode newDataTree,
-            final ResolveDataChangeEventsTask listenerResolver) {
-        LOG.debug("Updating Store snaphot version: {} with version:{}", currentSnapshot, newDataTree.getSubtreeVersion());
-
-        if (LOG.isTraceEnabled()) {
-            LOG.trace("Data Tree is {}", StoreUtils.toStringTree(newDataTree.getData()));
-        }
-
-        /*
-         * The commit has to occur atomically with regard to listener
-         * registrations.
-         */
-        synchronized (this) {
-            dataTree.commitSnapshot(currentSnapshot, newDataTree);
-
-            for (ChangeListenerNotifyTask task : listenerResolver.call()) {
-                LOG.trace("Scheduling invocation of listeners: {}", task);
-                executor.submit(task);
-            }
-        }
-    }
-
     private static abstract class AbstractDOMStoreTransaction implements DOMStoreTransaction {
         private final Object identifier;
 
@@ -198,9 +181,9 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
 
     private static final class SnapshotBackedReadTransaction extends AbstractDOMStoreTransaction implements
             DOMStoreReadTransaction {
-        private DataTree.Snapshot stableSnapshot;
+        private DataTreeSnapshot stableSnapshot;
 
-        public SnapshotBackedReadTransaction(final Object identifier, final DataTree.Snapshot snapshot) {
+        public SnapshotBackedReadTransaction(final Object identifier, final DataTreeSnapshot snapshot) {
             super(identifier);
             this.stableSnapshot = Preconditions.checkNotNull(snapshot);
             LOG.debug("ReadOnly Tx: {} allocated with snapshot {}", identifier, snapshot);
@@ -226,10 +209,10 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         private InMemoryDOMDataStore store;
         private boolean ready = false;
 
-        public SnapshotBackedWriteTransaction(final Object identifier, final DataTree.Snapshot snapshot,
+        public SnapshotBackedWriteTransaction(final Object identifier, final DataTreeSnapshot snapshot,
                 final InMemoryDOMDataStore store, final ModificationApplyOperation applyOper) {
             super(identifier);
-            mutableTree = DataTreeModification.from(snapshot, applyOper);
+            mutableTree = snapshot.newModification(applyOper);
             this.store = store;
             LOG.debug("Write Tx: {} allocated with snapshot {}", identifier, snapshot);
         }
@@ -308,7 +291,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
     private static class SnapshotBackedReadWriteTransaction extends SnapshotBackedWriteTransaction implements
             DOMStoreReadWriteTransaction {
 
-        protected SnapshotBackedReadWriteTransaction(final Object identifier, final DataTree.Snapshot snapshot,
+        protected SnapshotBackedReadWriteTransaction(final Object identifier, final DataTreeSnapshot snapshot,
                 final InMemoryDOMDataStore store, final ModificationApplyOperation applyOper) {
             super(identifier, snapshot, store, applyOper);
         }
@@ -317,7 +300,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         public ListenableFuture<Optional<NormalizedNode<?, ?>>> read(final InstanceIdentifier path) {
             LOG.trace("Tx: {} Read: {}", getIdentifier(), path);
             try {
-                return Futures.immediateFuture(getMutatedView().read(path));
+                return Futures.immediateFuture(getMutatedView().readNode(path));
             } catch (Exception e) {
                 LOG.error("Tx: {} Failed Read of {}", getIdentifier(), path, e);
                 throw e;
@@ -328,62 +311,47 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
     private class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort {
 
         private final SnapshotBackedWriteTransaction transaction;
-        private final NodeModification modification;
+        private final DataTreeModification modification;
 
-        private DataTree.Snapshot storeSnapshot;
-        private Optional<StoreMetadataNode> proposedSubtree;
         private ResolveDataChangeEventsTask listenerResolver;
+        private DataTreeCandidate candidate;
 
         public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction writeTransaction) {
             this.transaction = writeTransaction;
-            this.modification = transaction.getMutatedView().getRootModification();
+            this.modification = transaction.getMutatedView();
         }
 
         @Override
         public ListenableFuture<Boolean> canCommit() {
-            final DataTree.Snapshot snapshotCapture = dataTree.takeSnapshot();
-            final ModificationApplyOperation snapshotOperation = operationTree;
-
             return executor.submit(new Callable<Boolean>() {
-
                 @Override
-                public Boolean call() throws Exception {
-                    Boolean applicable = false;
+                public Boolean call() {
                     try {
-                        snapshotOperation.checkApplicable(PUBLIC_ROOT_PATH, modification,
-                            Optional.of(snapshotCapture.getRootNode()));
-                        applicable = true;
+                       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);
-                        applicable = false;
+                        return false;
                     }
-                    LOG.debug("Store Transaction: {} : canCommit : {}", transaction.getIdentifier(), applicable);
-                    return applicable;
                 }
             });
         }
 
         @Override
         public ListenableFuture<Void> preCommit() {
-            storeSnapshot = dataTree.takeSnapshot();
-            if (modification.getModificationType() == ModificationType.UNMODIFIED) {
-                return Futures.immediateFuture(null);
-            }
             return executor.submit(new Callable<Void>() {
-
                 @Override
-                public Void call() throws Exception {
-                    StoreMetadataNode metadataTree = storeSnapshot.getRootNode();
+                public Void call() {
+                       candidate = dataTree.prepare(modification);
 
-                    proposedSubtree = operationTree.apply(modification, Optional.of(metadataTree),
-                            increase(metadataTree.getSubtreeVersion()));
+                    listenerResolver = ResolveDataChangeEventsTask.create(candidate, listenerTree);
 
-                    listenerResolver = ResolveDataChangeEventsTask.create() //
-                            .setRootPath(PUBLIC_ROOT_PATH) //
-                            .setBeforeRoot(Optional.of(metadataTree)) //
-                            .setAfterRoot(proposedSubtree) //
-                            .setModificationRoot(modification) //
-                            .setListenerRoot(listenerTree);
+//                            .setRootPath(PUBLIC_ROOT_PATH) //
+//                            .setBeforeRoot(Optional.of(metadataTree)) //
+//                            .setAfterRoot(proposedSubtree) //
+//                            .setModificationRoot(modification.getRootModification()) //
+//                            .setListenerRoot(listenerTree);
 
                     return null;
                 }
@@ -392,24 +360,33 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
 
         @Override
         public ListenableFuture<Void> abort() {
-            storeSnapshot = null;
-            proposedSubtree = null;
+               if (candidate != null) {
+                       candidate.close();
+                       candidate = null;
+               }
+
             return Futures.<Void> 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(), listenerResolver);
             return Futures.<Void> immediateFuture(null);
         }
-
     }
 
     private static final class AlwaysFailOperation implements ModificationApplyOperation {
index 780291e70f3b1399be48648a8d2a451a8608a134..153df768ae7c4383c9a9cd399f3009537bb78456 100644 (file)
@@ -7,8 +7,9 @@
  */
 package org.opendaylight.controller.md.sal.dom.store.impl;
 
-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.ModificationApplyOperation;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.NodeModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.StoreMetadataNode;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 
index 44d50166af21df772f478d057fa479d879e0b829..db9bb0fef29fda971f08982d5d0f0e67353ec630 100644 (file)
@@ -8,7 +8,7 @@
 package org.opendaylight.controller.md.sal.dom.store.impl;
 
 import static org.opendaylight.controller.md.sal.dom.store.impl.DOMImmutableDataChangeEvent.builder;
-import static org.opendaylight.controller.md.sal.dom.store.impl.StoreUtils.append;
+import static org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreUtils.append;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -22,11 +22,12 @@ import java.util.concurrent.Callable;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataChangeScope;
 import org.opendaylight.controller.md.sal.dom.store.impl.DOMImmutableDataChangeEvent.Builder;
 import org.opendaylight.controller.md.sal.dom.store.impl.DOMImmutableDataChangeEvent.SimpleEventFactory;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeCandidate;
 import org.opendaylight.controller.md.sal.dom.store.impl.tree.ListenerTree;
 import org.opendaylight.controller.md.sal.dom.store.impl.tree.ListenerTree.Node;
 import org.opendaylight.controller.md.sal.dom.store.impl.tree.ListenerTree.Walker;
-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.data.NodeModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.StoreMetadataNode;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.NodeIdentifier;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.NodeIdentifierWithPredicates;
@@ -38,6 +39,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
@@ -62,61 +64,18 @@ import com.google.common.collect.Multimap;
  * </ul>
  *
  */
-public class ResolveDataChangeEventsTask implements Callable<Iterable<ChangeListenerNotifyTask>> {
+final class ResolveDataChangeEventsTask implements Callable<Iterable<ChangeListenerNotifyTask>> {
     private static final Logger LOG = LoggerFactory.getLogger(ResolveDataChangeEventsTask.class);
     private static final DOMImmutableDataChangeEvent NO_CHANGE = builder(DataChangeScope.BASE).build();
 
-    private InstanceIdentifier rootPath;
-    private ListenerTree listenerRoot;
-    private NodeModification modificationRoot;
-    private Optional<StoreMetadataNode> beforeRoot;
-    private Optional<StoreMetadataNode> afterRoot;
     private final Multimap<ListenerTree.Node, DOMImmutableDataChangeEvent> events = HashMultimap.create();
+    private final DataTreeCandidate candidate;
+    private final ListenerTree listenerRoot;
 
-    protected InstanceIdentifier getRootPath() {
-        return rootPath;
-    }
-
-    protected ResolveDataChangeEventsTask setRootPath(final InstanceIdentifier rootPath) {
-        this.rootPath = rootPath;
-        return this;
-    }
-
-    protected ListenerTree getListenerRoot() {
-        return listenerRoot;
-    }
-
-    protected ResolveDataChangeEventsTask setListenerRoot(final ListenerTree listenerRoot) {
-        this.listenerRoot = listenerRoot;
-        return this;
-    }
-
-    protected NodeModification getModificationRoot() {
-        return modificationRoot;
-    }
-
-    protected ResolveDataChangeEventsTask setModificationRoot(final NodeModification modificationRoot) {
-        this.modificationRoot = modificationRoot;
-        return this;
-    }
-
-    protected Optional<StoreMetadataNode> getBeforeRoot() {
-        return beforeRoot;
-    }
-
-    protected ResolveDataChangeEventsTask setBeforeRoot(final Optional<StoreMetadataNode> beforeRoot) {
-        this.beforeRoot = beforeRoot;
-        return this;
-    }
-
-    protected Optional<StoreMetadataNode> getAfterRoot() {
-        return afterRoot;
-    }
-
-    protected ResolveDataChangeEventsTask setAfterRoot(final Optional<StoreMetadataNode> afterRoot) {
-        this.afterRoot = afterRoot;
-        return this;
-    }
+    public ResolveDataChangeEventsTask(DataTreeCandidate candidate, ListenerTree listenerTree) {
+       this.candidate = Preconditions.checkNotNull(candidate);
+       this.listenerRoot = Preconditions.checkNotNull(listenerTree);
+       }
 
     /**
      * Resolves and creates Notification Tasks
@@ -129,11 +88,10 @@ public class ResolveDataChangeEventsTask implements Callable<Iterable<ChangeList
      */
     @Override
     public Iterable<ChangeListenerNotifyTask> call() {
-        LOG.trace("Resolving events for {}", modificationRoot);
-
         try (final Walker w = listenerRoot.getWalker()) {
-            resolveAnyChangeEvent(rootPath, Collections.singleton(w.getRootNode()), modificationRoot, beforeRoot,
-                    afterRoot);
+            resolveAnyChangeEvent(candidate.getRootPath(), Collections.singleton(w.getRootNode()),
+                       candidate.getModificationRoot(), Optional.fromNullable(candidate.getBeforeRoot()),
+                       Optional.fromNullable(candidate.getAfterRoot()));
             return createNotificationTasks();
         }
     }
@@ -556,7 +514,7 @@ public class ResolveDataChangeEventsTask implements Callable<Iterable<ChangeList
         }
     }
 
-    public static ResolveDataChangeEventsTask create() {
-        return new ResolveDataChangeEventsTask();
-    }
+       public static ResolveDataChangeEventsTask create(DataTreeCandidate candidate, ListenerTree listenerTree) {
+        return new ResolveDataChangeEventsTask(candidate, listenerTree);
+       }
 }
index afe9653394ec4a3363d34a48d9cd937b73363de1..57e97113ebe88943cbeff8fabb3c324e14596545 100644 (file)
@@ -15,10 +15,13 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 
-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.StoreNodeCompositeBuilder;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataPreconditionFailedException;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.ModificationApplyOperation;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreUtils;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.ModificationType;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.NodeModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.StoreMetadataNode;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.StoreNodeCompositeBuilder;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.AugmentationIdentifier;
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataPreconditionFailedException.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataPreconditionFailedException.java
new file mode 100644 (file)
index 0000000..bb2111e
--- /dev/null
@@ -0,0 +1,31 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+
+import com.google.common.base.Preconditions;
+
+public class DataPreconditionFailedException extends Exception {
+    private static final long serialVersionUID = 1L;
+    private final InstanceIdentifier path;
+
+    public DataPreconditionFailedException(final InstanceIdentifier path, final String message) {
+        super(message);
+        this.path = Preconditions.checkNotNull(path);
+    }
+
+    public DataPreconditionFailedException(final InstanceIdentifier path, final String message, final Throwable cause) {
+        super(message, cause);
+        this.path = Preconditions.checkNotNull(path);
+    }
+
+    public InstanceIdentifier getPath() {
+        return path;
+    }
+}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTree.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTree.java
new file mode 100644 (file)
index 0000000..01e2a29
--- /dev/null
@@ -0,0 +1,51 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+
+/**
+ * Interface representing a data tree which can be modified in an MVCC fashion.
+ */
+public interface DataTree {
+       /**
+        * Take a read-only point-in-time snapshot of the tree.
+        *
+        * @return Data tree snapshot.
+        */
+       DataTreeSnapshot takeSnapshot();
+
+       /**
+        * Make the data tree use a new schema context. The context will be used
+        * only by subsequent operations.
+        *
+        * @param newSchemaContext new SchemaContext
+        * @throws IllegalArgumentException if the new context is incompatible
+        */
+    void setSchemaContext(SchemaContext newSchemaContext);
+
+    /**
+     * Validate whether a particular modification can be applied to the data tree.
+     */
+    void validate(DataTreeModification modification) throws DataPreconditionFailedException;
+
+    /**
+     * Prepare a modification for commit.
+     *
+     * @param modification
+     * @return candidate data tree
+     */
+    DataTreeCandidate prepare(DataTreeModification modification);
+
+    /**
+     * Commit a data tree candidate.
+     *
+     * @param candidate data tree candidate
+     */
+    void commit(DataTreeCandidate candidate);
+}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeCandidate.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeCandidate.java
new file mode 100644 (file)
index 0000000..79ce37e
--- /dev/null
@@ -0,0 +1,28 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.NodeModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.StoreMetadataNode;
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+
+public interface DataTreeCandidate extends AutoCloseable {
+       @Override
+       void close();
+
+       InstanceIdentifier getRootPath();
+
+       @Deprecated
+       NodeModification getModificationRoot();
+
+       @Deprecated
+       StoreMetadataNode getBeforeRoot();
+
+       @Deprecated
+       StoreMetadataNode getAfterRoot();
+}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeFactory.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeFactory.java
new file mode 100644 (file)
index 0000000..7422c11
--- /dev/null
@@ -0,0 +1,20 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+
+/**
+ * Factory interface for creating data trees.
+ */
+public interface DataTreeFactory {
+       /**
+        * Create a new data tree.
+        *
+        * @return A data tree instance.
+        */
+       DataTree create();
+}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeModification.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeModification.java
new file mode 100644 (file)
index 0000000..cff90a4
--- /dev/null
@@ -0,0 +1,23 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+
+/**
+ * Class encapsulation of set of modifications to a base tree. This tree is backed
+ * by a read-only snapshot and tracks modifications on top of that. The modification
+ * has the ability to rebase itself to a new snapshot.
+ */
+public interface DataTreeModification extends DataTreeSnapshot {
+       void delete(InstanceIdentifier path);
+       void merge(InstanceIdentifier path, NormalizedNode<?, ?> data);
+       void write(InstanceIdentifier path, NormalizedNode<?, ?> data);
+       void seal();
+}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeSnapshot.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeSnapshot.java
new file mode 100644 (file)
index 0000000..4f35128
--- /dev/null
@@ -0,0 +1,37 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+
+import com.google.common.base.Optional;
+
+/**
+ * Read-only snapshot of a {@link DataTree}. The snapshot is stable and isolated,
+ * e.g. data tree changes occurring after the snapshot has been taken are not
+ * visible through the snapshot.
+ */
+public interface DataTreeSnapshot {
+       /**
+        * Read a particular node from the snapshot.
+        *
+        * @param path Path of the node
+        * @return Optional result encapsulating the presence and value of the node
+        */
+    Optional<NormalizedNode<?, ?>> readNode(InstanceIdentifier path);
+
+    /**
+     * Create a new data tree modification based on this snapshot, using the
+     * specified data application strategy.
+     *
+     * @param strategy data modification strategy
+     * @return A new data tree modification
+     */
+    DataTreeModification newModification(ModificationApplyOperation applyOper);
+}
@@ -5,11 +5,10 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-package org.opendaylight.controller.md.sal.dom.store.impl;
+package org.opendaylight.controller.md.sal.dom.store.impl.tree;
 
-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.StoreTreeNode;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.NodeModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.StoreMetadataNode;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
 
@@ -5,7 +5,7 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-package org.opendaylight.controller.md.sal.dom.store.impl;
+package org.opendaylight.controller.md.sal.dom.store.impl.tree;
 
 import java.util.Set;
 
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/AbstractDataTreeCandidate.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/AbstractDataTreeCandidate.java
new file mode 100644 (file)
index 0000000..b2faf79
--- /dev/null
@@ -0,0 +1,33 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
+
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeCandidate;
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+
+import com.google.common.base.Preconditions;
+
+abstract class AbstractDataTreeCandidate implements DataTreeCandidate {
+       private final InstanceIdentifier rootPath;
+       private final NodeModification modificationRoot;
+
+       protected AbstractDataTreeCandidate(final InstanceIdentifier rootPath, NodeModification modificationRoot) {
+               this.rootPath = Preconditions.checkNotNull(rootPath);
+               this.modificationRoot = Preconditions.checkNotNull(modificationRoot);
+       }
+
+       @Override
+       public final InstanceIdentifier getRootPath() {
+               return rootPath;
+       }
+
+       @Override
+       public final NodeModification getModificationRoot() {
+               return modificationRoot;
+       }
+}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTree.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTree.java
new file mode 100644 (file)
index 0000000..1f2a775
--- /dev/null
@@ -0,0 +1,127 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+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.StoreUtils;
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+
+/**
+ * Read-only snapshot of the data tree.
+ */
+final class InMemoryDataTree implements DataTree {
+    private static final Logger LOG = LoggerFactory.getLogger(InMemoryDataTree.class);
+    private static final InstanceIdentifier PUBLIC_ROOT_PATH = InstanceIdentifier.builder().build();
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock(true);
+    private StoreMetadataNode rootNode;
+    private SchemaContext currentSchemaContext;
+
+    public InMemoryDataTree(StoreMetadataNode rootNode, final SchemaContext schemaContext) {
+        this.rootNode = Preconditions.checkNotNull(rootNode);
+        this.currentSchemaContext = schemaContext;
+    }
+
+    @Override
+       public synchronized void setSchemaContext(final SchemaContext newSchemaContext) {
+        Preconditions.checkNotNull(newSchemaContext);
+
+        LOG.info("Attepting to install schema context {}", newSchemaContext);
+
+        /*
+         * FIXME: we should walk the schema contexts, both current and new and see
+         *        whether they are compatible here. Reject incompatible changes.
+         */
+
+        // Ready to change the context now, make sure no operations are running
+        rwLock.writeLock().lock();
+        try {
+            this.currentSchemaContext = newSchemaContext;
+        } finally {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+       public InMemoryDataTreeSnapshot takeSnapshot() {
+        rwLock.readLock().lock();
+        try {
+            return new InMemoryDataTreeSnapshot(currentSchemaContext, rootNode);
+        } finally {
+            rwLock.readLock().unlock();
+        }
+    }
+
+       @Override
+       public void validate(DataTreeModification modification) throws DataPreconditionFailedException {
+               Preconditions.checkArgument(modification instanceof InMemoryDataTreeModification, "Invalid modification class %s", modification.getClass());
+
+               final InMemoryDataTreeModification m = (InMemoryDataTreeModification)modification;
+               m.getStrategy().checkApplicable(PUBLIC_ROOT_PATH, m.getRootModification(), Optional.of(rootNode));
+       }
+
+       @Override
+       public synchronized DataTreeCandidate prepare(DataTreeModification modification) {
+               Preconditions.checkArgument(modification instanceof InMemoryDataTreeModification, "Invalid modification class %s", modification.getClass());
+
+               final InMemoryDataTreeModification m = (InMemoryDataTreeModification)modification;
+               final NodeModification root = m.getRootModification();
+
+        if (root.getModificationType() == ModificationType.UNMODIFIED) {
+               return new NoopDataTreeCandidate(PUBLIC_ROOT_PATH, root);
+        }
+
+        rwLock.writeLock().lock();
+        try {
+               // FIXME: rootNode needs to be a read-write snapshot here...
+               final Optional<StoreMetadataNode> newRoot = m.getStrategy().apply(m.getRootModification(), Optional.of(rootNode), StoreUtils.increase(rootNode.getSubtreeVersion()));
+               Preconditions.checkState(newRoot.isPresent(), "Apply strategy failed to produce root node");
+               return new InMemoryDataTreeCandidate(PUBLIC_ROOT_PATH, root, rootNode, newRoot.get());
+        } finally {
+               rwLock.writeLock().unlock();
+        }
+       }
+
+       @Override
+       public synchronized void commit(DataTreeCandidate candidate) {
+               if (candidate instanceof NoopDataTreeCandidate) {
+                       return;
+               }
+
+               Preconditions.checkArgument(candidate instanceof InMemoryDataTreeCandidate, "Invalid candidate class %s", candidate.getClass());
+               final InMemoryDataTreeCandidate c = (InMemoryDataTreeCandidate)candidate;
+
+        LOG.debug("Updating Store snapshot version: {} with version:{}", rootNode.getSubtreeVersion(), c.getAfterRoot().getSubtreeVersion());
+
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("Data Tree is {}", StoreUtils.toStringTree(c.getAfterRoot().getData()));
+        }
+
+        // Ready to change the context now, make sure no operations are running
+        rwLock.writeLock().lock();
+        try {
+            Preconditions.checkState(c.getBeforeRoot() == rootNode,
+                    String.format("Store snapshot %s and transaction snapshot %s differ.", rootNode, c.getBeforeRoot()));
+            this.rootNode = c.getAfterRoot();
+        } finally {
+            rwLock.writeLock().unlock();
+        }
+       }
+}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeCandidate.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeCandidate.java
new file mode 100644 (file)
index 0000000..93719b7
--- /dev/null
@@ -0,0 +1,32 @@
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
+
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+
+import com.google.common.base.Preconditions;
+
+final class InMemoryDataTreeCandidate extends AbstractDataTreeCandidate {
+       private final StoreMetadataNode newRoot;
+       private final StoreMetadataNode oldRoot;
+
+       InMemoryDataTreeCandidate(final InstanceIdentifier rootPath, final NodeModification modificationRoot,
+                       final StoreMetadataNode oldRoot, final StoreMetadataNode newRoot) {
+               super(rootPath, modificationRoot);
+               this.newRoot = Preconditions.checkNotNull(newRoot);
+               this.oldRoot = Preconditions.checkNotNull(oldRoot);
+       }
+
+       @Override
+       public void close() {
+               // FIXME: abort operation here :)
+       }
+
+       @Override
+       public StoreMetadataNode getBeforeRoot() {
+               return oldRoot;
+       }
+
+       @Override
+       public StoreMetadataNode getAfterRoot() {
+               return newRoot;
+       }
+}
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeFactory.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeFactory.java
new file mode 100644 (file)
index 0000000..7614611
--- /dev/null
@@ -0,0 +1,35 @@
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
+
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeFactory;
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.NodeIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.impl.schema.Builders;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+
+/**
+ * A factory for creating in-memory data trees.
+ */
+public final class InMemoryDataTreeFactory implements DataTreeFactory {
+       private static final InMemoryDataTreeFactory INSTANCE = new InMemoryDataTreeFactory();
+
+       private InMemoryDataTreeFactory() {
+               // Never instantiated externally
+       }
+
+       @Override
+       public InMemoryDataTree create() {
+               final NodeIdentifier root = new NodeIdentifier(SchemaContext.NAME);
+               final NormalizedNode<?, ?> data = Builders.containerBuilder().withNodeIdentifier(root).build();
+
+               return new InMemoryDataTree(StoreMetadataNode.createEmpty(data), null);
+       }
+
+       /**
+        * Get an instance of this factory. This method cannot fail.
+        *
+        * @return Data tree factory instance.
+        */
+       public static final InMemoryDataTreeFactory getInstance() {
+               return INSTANCE;
+       }
+}
@@ -5,15 +5,17 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-package org.opendaylight.controller.md.sal.dom.store.impl;
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
 
 import static com.google.common.base.Preconditions.checkState;
 
 import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
-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.OperationWithModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeModification;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.ModificationApplyOperation;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreUtils;
 import org.opendaylight.controller.md.sal.dom.store.impl.tree.TreeNodeUtils;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
@@ -26,38 +28,43 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 
-/**
- * Class encapsulation of set of modifications to a base tree. This tree is backed
- * by a read-only snapshot and tracks modifications on top of that. The modification
- * has the ability to rebase itself on a new snapshot.
- */
-class DataTreeModification {
-    private static final Logger LOG = LoggerFactory.getLogger(DataTreeModification.class);
+final class InMemoryDataTreeModification implements DataTreeModification {
+    private static final Logger LOG = LoggerFactory.getLogger(InMemoryDataTreeModification.class);
 
     /*
      * FIXME: the thread safety of concurrent write/delete/read/seal operations
      *        needs to be evaluated.
      */
-    private static final AtomicIntegerFieldUpdater<DataTreeModification> SEALED_UPDATER =
-            AtomicIntegerFieldUpdater.newUpdater(DataTreeModification.class, "sealed");
+    private static final AtomicIntegerFieldUpdater<InMemoryDataTreeModification> SEALED_UPDATER =
+            AtomicIntegerFieldUpdater.newUpdater(InMemoryDataTreeModification.class, "sealed");
     private volatile int sealed = 0;
 
     private final ModificationApplyOperation strategyTree;
-    private final DataTree.Snapshot snapshot;
+    private final InMemoryDataTreeSnapshot snapshot;
     private final NodeModification rootNode;
 
-    private DataTreeModification(final DataTree.Snapshot snapshot, final ModificationApplyOperation strategyTree) {
+    InMemoryDataTreeModification(final InMemoryDataTreeSnapshot snapshot, final ModificationApplyOperation resolver) {
         this.snapshot = Preconditions.checkNotNull(snapshot);
-        this.strategyTree = Preconditions.checkNotNull(strategyTree);
+        this.strategyTree = Preconditions.checkNotNull(resolver);
         this.rootNode = NodeModification.createUnmodified(snapshot.getRootNode());
     }
 
-    public void write(final InstanceIdentifier path, final NormalizedNode<?, ?> value) {
+    NodeModification getRootModification() {
+        return rootNode;
+    }
+
+       ModificationApplyOperation getStrategy() {
+               return strategyTree;
+       }
+
+    @Override
+       public void write(final InstanceIdentifier path, final NormalizedNode<?, ?> value) {
         checkSealed();
         resolveModificationFor(path).write(value);
     }
 
-    public void merge(final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
+    @Override
+       public void merge(final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
         checkSealed();
         mergeImpl(resolveModificationFor(path),data);
     }
@@ -75,12 +82,14 @@ class DataTreeModification {
         op.merge(data);
     }
 
-    public void delete(final InstanceIdentifier path) {
+    @Override
+       public void delete(final InstanceIdentifier path) {
         checkSealed();
         resolveModificationFor(path).delete();
     }
 
-    public Optional<NormalizedNode<?, ?>> read(final InstanceIdentifier path) {
+    @Override
+       public Optional<NormalizedNode<?, ?>> readNode(final InstanceIdentifier path) {
         Entry<InstanceIdentifier, NodeModification> modification = TreeNodeUtils.findClosestsOrFirstMatch(rootNode, path, NodeModification.IS_TERMINAL_PREDICATE);
 
         Optional<StoreMetadataNode> result = resolveSnapshot(modification);
@@ -128,11 +137,8 @@ class DataTreeModification {
         return OperationWithModification.from(operation, modification);
     }
 
-    public static DataTreeModification from(final DataTree.Snapshot snapshot, final ModificationApplyOperation resolver) {
-        return new DataTreeModification(snapshot, resolver);
-    }
-
-    public void seal() {
+    @Override
+       public void seal() {
         final boolean success = SEALED_UPDATER.compareAndSet(this, 0, 1);
         Preconditions.checkState(success, "Attempted to seal an already-sealed Data Tree.");
         rootNode.seal();
@@ -142,13 +148,14 @@ class DataTreeModification {
         checkState(sealed == 0, "Data Tree is sealed. No further modifications allowed.");
     }
 
-    @Deprecated
-    protected NodeModification getRootModification() {
-        return rootNode;
-    }
-
     @Override
     public String toString() {
         return "MutableDataTree [modification=" + rootNode + "]";
     }
+
+       @Override
+       public DataTreeModification newModification(ModificationApplyOperation applyOper) {
+       // FIXME: transaction chaining
+       throw new UnsupportedOperationException("Implement this as part of transaction chaining");
+       }
 }
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeSnapshot.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeSnapshot.java
new file mode 100644 (file)
index 0000000..96f1565
--- /dev/null
@@ -0,0 +1,45 @@
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
+
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeSnapshot;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.ModificationApplyOperation;
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+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 com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+
+final class InMemoryDataTreeSnapshot implements DataTreeSnapshot {
+    private final SchemaContext schemaContext;
+    private final StoreMetadataNode rootNode;
+
+    InMemoryDataTreeSnapshot(final SchemaContext schemaContext, final StoreMetadataNode rootNode) {
+        this.schemaContext = Preconditions.checkNotNull(schemaContext);
+        this.rootNode = Preconditions.checkNotNull(rootNode);
+    }
+
+       StoreMetadataNode getRootNode() {
+        return rootNode;
+    }
+
+    SchemaContext getSchemaContext() {
+        return schemaContext;
+    }
+
+    @Override
+       public Optional<NormalizedNode<?, ?>> readNode(final InstanceIdentifier path) {
+        return NormalizedNodeUtils.findNode(rootNode.getData(), path);
+    }
+
+    @Override
+       public InMemoryDataTreeModification newModification(ModificationApplyOperation applyOper) {
+               return new InMemoryDataTreeModification(this, applyOper);
+       }
+
+    @Override
+    public String toString() {
+        return rootNode.getSubtreeVersion().toString();
+    }
+
+}
\ No newline at end of file
@@ -5,7 +5,7 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
 
 public enum ModificationType {
 
@@ -5,7 +5,7 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
 
 import static com.google.common.base.Preconditions.checkState;
 
@@ -14,6 +14,7 @@ import java.util.Map;
 
 import javax.annotation.concurrent.GuardedBy;
 
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreTreeNode;
 import org.opendaylight.yangtools.concepts.Identifiable;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/NoopDataTreeCandidate.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/NoopDataTreeCandidate.java
new file mode 100644 (file)
index 0000000..1782da2
--- /dev/null
@@ -0,0 +1,31 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
+
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+
+final class NoopDataTreeCandidate extends AbstractDataTreeCandidate {
+       protected NoopDataTreeCandidate(final InstanceIdentifier rootPath, final NodeModification modificationRoot) {
+               super(rootPath, modificationRoot);
+       }
+
+       @Override
+       public void close() {
+               // NO-OP
+       }
+
+       @Override
+       public StoreMetadataNode getBeforeRoot() {
+               return null;
+       }
+
+       @Override
+       public StoreMetadataNode getAfterRoot() {
+               return null;
+       }
+}
@@ -5,7 +5,7 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
 
 import static com.google.common.base.Preconditions.checkState;
 
@@ -13,6 +13,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreTreeNode;
 import org.opendaylight.yangtools.concepts.Identifiable;
 import org.opendaylight.yangtools.concepts.Immutable;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
@@ -5,7 +5,7 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-package org.opendaylight.controller.md.sal.dom.store.impl.tree;
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
 
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.impl.schema.builder.api.NormalizedNodeContainerBuilder;
@@ -1,4 +1,11 @@
-package org.opendaylight.controller.md.sal.dom.store.impl;
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -19,7 +26,10 @@ import static org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes.ma
 
 import org.junit.Before;
 import org.junit.Test;
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreMetadataNode;
+import org.opendaylight.controller.md.sal.dom.store.impl.SchemaAwareApplyOperationRoot;
+import org.opendaylight.controller.md.sal.dom.store.impl.TestModel;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTree;
+import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeModification;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.NodeIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
@@ -137,10 +147,10 @@ public class ModificationMetadataTreeTest {
 
     @Test
     public void basicReadWrites() {
-        DataTreeModification modificationTree = DataTreeModification.from(new DataTree.Snapshot(schemaContext,
+        DataTreeModification modificationTree = new InMemoryDataTreeModification(new InMemoryDataTreeSnapshot(schemaContext,
                 StoreMetadataNode.createRecursively(createDocumentOne(), UnsignedLong.valueOf(5))),
                 new SchemaAwareApplyOperationRoot(schemaContext));
-        Optional<NormalizedNode<?, ?>> originalBarNode = modificationTree.read(OUTER_LIST_2_PATH);
+        Optional<NormalizedNode<?, ?>> originalBarNode = modificationTree.readNode(OUTER_LIST_2_PATH);
         assertTrue(originalBarNode.isPresent());
         assertSame(BAR_NODE, originalBarNode.get());
 
@@ -149,13 +159,13 @@ public class ModificationMetadataTreeTest {
 
         // reads node to /outer-list/1/inner_list/two/value
         // and checks if node is already present
-        Optional<NormalizedNode<?, ?>> barTwoCModified = modificationTree.read(TWO_TWO_VALUE_PATH);
+        Optional<NormalizedNode<?, ?>> barTwoCModified = modificationTree.readNode(TWO_TWO_VALUE_PATH);
         assertTrue(barTwoCModified.isPresent());
         assertEquals(ImmutableNodes.leafNode(VALUE_QNAME, "test"), barTwoCModified.get());
 
         // delete node to /outer-list/1/inner_list/two/value
         modificationTree.delete(TWO_TWO_VALUE_PATH);
-        Optional<NormalizedNode<?, ?>> barTwoCAfterDelete = modificationTree.read(TWO_TWO_VALUE_PATH);
+        Optional<NormalizedNode<?, ?>> barTwoCAfterDelete = modificationTree.readNode(TWO_TWO_VALUE_PATH);
         assertFalse(barTwoCAfterDelete.isPresent());
     }
 
@@ -164,7 +174,8 @@ public class ModificationMetadataTreeTest {
         /**
          * Creates empty Snapshot with associated schema context.
          */
-        DataTree t = DataTree.create(schemaContext);
+        DataTree t = InMemoryDataTreeFactory.getInstance().create();
+        t.setSchemaContext(schemaContext);
 
         /**
          *
@@ -172,9 +183,7 @@ public class ModificationMetadataTreeTest {
          * context.
          *
          */
-        DataTreeModification modificationTree = DataTreeModification.from(t.takeSnapshot(), new SchemaAwareApplyOperationRoot(
-                schemaContext));
-        return modificationTree;
+        return t.takeSnapshot().newModification(new SchemaAwareApplyOperationRoot(schemaContext));
     }
 
     @Test
@@ -195,14 +204,14 @@ public class ModificationMetadataTreeTest {
         /**
          * Reads list node from /test/outer-list
          */
-        Optional<NormalizedNode<?, ?>> potentialOuterList = modificationTree.read(OUTER_LIST_PATH);
+        Optional<NormalizedNode<?, ?>> potentialOuterList = modificationTree.readNode(OUTER_LIST_PATH);
         assertTrue(potentialOuterList.isPresent());
 
         /**
          * Reads container node from /test and verifies that it contains test
          * node
          */
-        Optional<NormalizedNode<?, ?>> potentialTest = modificationTree.read(TEST_PATH);
+        Optional<NormalizedNode<?, ?>> potentialTest = modificationTree.readNode(TEST_PATH);
         ContainerNode containerTest = assertPresentAndType(potentialTest, ContainerNode.class);
 
         /**
@@ -219,8 +228,8 @@ public class ModificationMetadataTreeTest {
     public void writeSubtreeReadChildren() {
         DataTreeModification modificationTree = createEmptyModificationTree();
         modificationTree.write(TEST_PATH, createTestContainer());
-        Optional<NormalizedNode<?, ?>> potential = modificationTree.read(TWO_TWO_PATH);
-        MapEntryNode node = assertPresentAndType(potential, MapEntryNode.class);
+        Optional<NormalizedNode<?, ?>> potential = modificationTree.readNode(TWO_TWO_PATH);
+        assertPresentAndType(potential, MapEntryNode.class);
     }
 
     @Test
@@ -229,11 +238,11 @@ public class ModificationMetadataTreeTest {
         modificationTree.write(TEST_PATH, createTestContainer());
 
         // We verify data are present
-        Optional<NormalizedNode<?, ?>> potentialBeforeDelete = modificationTree.read(TWO_TWO_PATH);
-        MapEntryNode node = assertPresentAndType(potentialBeforeDelete, MapEntryNode.class);
+        Optional<NormalizedNode<?, ?>> potentialBeforeDelete = modificationTree.readNode(TWO_TWO_PATH);
+        assertPresentAndType(potentialBeforeDelete, MapEntryNode.class);
 
         modificationTree.delete(TWO_TWO_PATH);
-        Optional<NormalizedNode<?, ?>> potentialAfterDelete = modificationTree.read(TWO_TWO_PATH);
+        Optional<NormalizedNode<?, ?>> potentialAfterDelete = modificationTree.readNode(TWO_TWO_PATH);
         assertFalse(potentialAfterDelete.isPresent());
 
     }