Merge "Bug 953 - Change the BindingIndependentConnector to handle RpcInvocationStrate...
authorTony Tkacik <ttkacik@cisco.com>
Tue, 20 May 2014 14:57:56 +0000 (14:57 +0000)
committerGerrit Code Review <gerrit@opendaylight.org>
Tue, 20 May 2014 14:57:56 +0000 (14:57 +0000)
28 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/ResolveDataChangeEventsTask.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/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/AlwaysFailOperation.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/MutableDataTree.java with 69% 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/ModificationApplyOperation.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationApplyOperation.java with 83% similarity]
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 96% 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/OperationWithModification.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/OperationWithModification.java with 89% similarity]
opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/SchemaAwareApplyOperation.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperation.java with 93% similarity]
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 96% 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 81% similarity]
opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/SchemaAwareApplyOperationRoot.java [moved from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperationRoot.java with 95% 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 7d2ff30b1fe64f96d89619894e83d26880f8bd73..9e11fc70fc066164585709200dee863c95d1600f 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,13 @@ 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.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;
@@ -31,7 +33,6 @@ 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.model.api.SchemaContext;
 import org.opendaylight.yangtools.yang.model.api.SchemaContextListener;
@@ -42,22 +43,17 @@ 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 ListenerTree listenerTree = ListenerTree.create();
-    private final DataTree dataTree = DataTree.create(null);
-    private ModificationApplyOperation operationTree = new AlwaysFailOperation();
 
     public InMemoryDOMDataStore(final String name, final ListeningExecutorService executor) {
         this.name = Preconditions.checkNotNull(name);
@@ -76,25 +72,17 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
 
     @Override
     public DOMStoreReadWriteTransaction newReadWriteTransaction() {
-        return new SnapshotBackedReadWriteTransaction(nextIdentifier(), dataTree.takeSnapshot(), this, operationTree);
+        return new SnapshotBackedReadWriteTransaction(nextIdentifier(), dataTree.takeSnapshot(), this);
     }
 
     @Override
     public DOMStoreWriteTransaction newWriteOnlyTransaction() {
-        return new SnapshotBackedWriteTransaction(nextIdentifier(), dataTree.takeSnapshot(), this, operationTree);
+        return new SnapshotBackedWriteTransaction(nextIdentifier(), dataTree.takeSnapshot(), this);
     }
 
     @Override
     public synchronized void onGlobalContextUpdated(final SchemaContext ctx) {
-        /*
-         * Order of operations is important: dataTree may reject the context
-         * and creation of ModificationApplyOperation may fail. So pre-construct
-         * the operation, then update the data tree and then move the operation
-         * into view.
-         */
-        final ModificationApplyOperation newOperationTree = SchemaAwareApplyOperationRoot.from(ctx);
         dataTree.setSchemaContext(ctx);
-        operationTree = newOperationTree;
     }
 
     @Override
@@ -145,28 +133,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;
 
@@ -197,10 +163,10 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
     }
 
     private static final class SnapshotBackedReadTransaction extends AbstractDOMStoreTransaction implements
-            DOMStoreReadTransaction {
-        private DataTree.Snapshot stableSnapshot;
+    DOMStoreReadTransaction {
+        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);
@@ -221,15 +187,15 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
     }
 
     private static class SnapshotBackedWriteTransaction extends AbstractDOMStoreTransaction implements
-            DOMStoreWriteTransaction {
-        private MutableDataTree mutableTree;
+    DOMStoreWriteTransaction {
+        private DataTreeModification mutableTree;
         private InMemoryDOMDataStore store;
         private boolean ready = false;
 
-        public SnapshotBackedWriteTransaction(final Object identifier, final DataTree.Snapshot snapshot,
-                final InMemoryDOMDataStore store, final ModificationApplyOperation applyOper) {
+        public SnapshotBackedWriteTransaction(final Object identifier, final DataTreeSnapshot snapshot,
+                final InMemoryDOMDataStore store) {
             super(identifier);
-            mutableTree = MutableDataTree.from(snapshot, applyOper);
+            mutableTree = snapshot.newModification();
             this.store = store;
             LOG.debug("Write Tx: {} allocated with snapshot {}", identifier, snapshot);
         }
@@ -295,7 +261,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
             return store.submit(this);
         }
 
-        protected MutableDataTree getMutatedView() {
+        protected DataTreeModification getMutatedView() {
             return mutableTree;
         }
 
@@ -306,18 +272,18 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
     }
 
     private static class SnapshotBackedReadWriteTransaction extends SnapshotBackedWriteTransaction implements
-            DOMStoreReadWriteTransaction {
+    DOMStoreReadWriteTransaction {
 
-        protected SnapshotBackedReadWriteTransaction(final Object identifier, final DataTree.Snapshot 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) {
             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,63 +294,40 @@ 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();
-
-                    proposedSubtree = operationTree.apply(modification, Optional.of(metadataTree),
-                            increase(metadataTree.getSubtreeVersion()));
-
-                    listenerResolver = ResolveDataChangeEventsTask.create() //
-                            .setRootPath(PUBLIC_ROOT_PATH) //
-                            .setBeforeRoot(Optional.of(metadataTree)) //
-                            .setAfterRoot(proposedSubtree) //
-                            .setModificationRoot(modification) //
-                            .setListenerRoot(listenerTree);
-
+                public Void call() {
+                    candidate = dataTree.prepare(modification);
+                    listenerResolver = ResolveDataChangeEventsTask.create(candidate, listenerTree);
                     return null;
                 }
             });
@@ -392,48 +335,32 @@ 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 {
-
-        @Override
-        public Optional<StoreMetadataNode> apply(final NodeModification modification,
-                final Optional<StoreMetadataNode> storeMeta, final UnsignedLong subtreeVersion) {
-            throw new IllegalStateException("Schema Context is not available.");
-        }
-
-        @Override
-        public void checkApplicable(final InstanceIdentifier path,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.");
-        }
-
     }
 }
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);
+       }
 }
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..a94acc5
--- /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();
+}
@@ -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/AlwaysFailOperation.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/AlwaysFailOperation.java
new file mode 100644 (file)
index 0000000..4e3aa49
--- /dev/null
@@ -0,0 +1,31 @@
+package org.opendaylight.controller.md.sal.dom.store.impl.tree.data;
+
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
+
+import com.google.common.base.Optional;
+import com.google.common.primitives.UnsignedLong;
+
+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.");
+    }
+
+    @Override
+    public void checkApplicable(final InstanceIdentifier path,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.");
+    }
+}
\ No newline at end of file
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..5a300a0
--- /dev/null
@@ -0,0 +1,136 @@
+/*
+ * 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 ModificationApplyOperation applyOper = new AlwaysFailOperation();
+    private SchemaContext currentSchemaContext;
+    private StoreMetadataNode rootNode;
+
+    public InMemoryDataTree(StoreMetadataNode rootNode, final SchemaContext schemaContext) {
+        this.rootNode = Preconditions.checkNotNull(rootNode);
+
+        if (schemaContext != null) {
+            // Also sets applyOper
+            setSchemaContext(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.
+         */
+
+        // Instantiate new apply operation, this still may fail
+        final ModificationApplyOperation newApplyOper = SchemaAwareApplyOperation.from(newSchemaContext);
+
+        // Ready to change the context now, make sure no operations are running
+        rwLock.writeLock().lock();
+        try {
+            this.applyOper = newApplyOper;
+            this.currentSchemaContext = newSchemaContext;
+        } finally {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public InMemoryDataTreeSnapshot takeSnapshot() {
+        rwLock.readLock().lock();
+        try {
+            return new InMemoryDataTreeSnapshot(currentSchemaContext, rootNode, applyOper);
+        } 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,15 @@
  * 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.AtomicBoolean;
+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.tree.DataTreeModification;
+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,28 +26,42 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 
-/*
- * FIXME: the thread safety of concurrent write/delete/read/seal operations
- *        needs to be evaluated.
- */
-class MutableDataTree {
-    private static final Logger LOG = LoggerFactory.getLogger(MutableDataTree.class);
-    private final AtomicBoolean sealed = new AtomicBoolean();
+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<InMemoryDataTreeModification> SEALED_UPDATER =
+            AtomicIntegerFieldUpdater.newUpdater(InMemoryDataTreeModification.class, "sealed");
+    private volatile int sealed = 0;
+
     private final ModificationApplyOperation strategyTree;
-    private final NodeModification rootModification;
-    private final DataTree.Snapshot snapshot;
+    private final InMemoryDataTreeSnapshot snapshot;
+    private final NodeModification rootNode;
 
-    private MutableDataTree(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.rootModification = NodeModification.createUnmodified(snapshot.getRootNode());
+        this.strategyTree = Preconditions.checkNotNull(resolver);
+        this.rootNode = NodeModification.createUnmodified(snapshot.getRootNode());
     }
 
+    NodeModification getRootModification() {
+        return rootNode;
+    }
+
+    ModificationApplyOperation getStrategy() {
+        return strategyTree;
+    }
+
+    @Override
     public void write(final InstanceIdentifier path, final NormalizedNode<?, ?> value) {
         checkSealed();
         resolveModificationFor(path).write(value);
     }
 
+    @Override
     public void merge(final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
         checkSealed();
         mergeImpl(resolveModificationFor(path),data);
@@ -66,13 +80,15 @@ class MutableDataTree {
         op.merge(data);
     }
 
+    @Override
     public void delete(final InstanceIdentifier path) {
         checkSealed();
         resolveModificationFor(path).delete();
     }
 
-    public Optional<NormalizedNode<?, ?>> read(final InstanceIdentifier path) {
-        Entry<InstanceIdentifier, NodeModification> modification = TreeNodeUtils.findClosestsOrFirstMatch(rootModification, path, NodeModification.IS_TERMINAL_PREDICATE);
+    @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);
         if (result.isPresent()) {
@@ -110,7 +126,7 @@ class MutableDataTree {
     }
 
     private OperationWithModification resolveModificationFor(final InstanceIdentifier path) {
-        NodeModification modification = rootModification;
+        NodeModification modification = rootNode;
         // We ensure strategy is present.
         ModificationApplyOperation operation = resolveModificationStrategy(path);
         for (PathArgument pathArg : path.getPath()) {
@@ -119,26 +135,25 @@ class MutableDataTree {
         return OperationWithModification.from(operation, modification);
     }
 
-    public static MutableDataTree from(final DataTree.Snapshot snapshot, final ModificationApplyOperation resolver) {
-        return new MutableDataTree(snapshot, resolver);
-    }
-
+    @Override
     public void seal() {
-        final boolean success = sealed.compareAndSet(false, true);
+        final boolean success = SEALED_UPDATER.compareAndSet(this, 0, 1);
         Preconditions.checkState(success, "Attempted to seal an already-sealed Data Tree.");
-        rootModification.seal();
+        rootNode.seal();
     }
 
     private void checkSealed() {
-        checkState(!sealed.get(), "Data Tree is sealed. No further modifications allowed.");
+        checkState(sealed == 0, "Data Tree is sealed. No further modifications allowed.");
     }
 
-    protected NodeModification getRootModification() {
-        return rootModification;
+    @Override
+    public String toString() {
+        return "MutableDataTree [modification=" + rootNode + "]";
     }
 
     @Override
-    public String toString() {
-        return "MutableDataTree [modification=" + rootModification + "]";
+    public DataTreeModification newModification() {
+        // 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..ce2d8c9
--- /dev/null
@@ -0,0 +1,47 @@
+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.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 ModificationApplyOperation applyOper;
+    private final SchemaContext schemaContext;
+    private final StoreMetadataNode rootNode;
+
+    InMemoryDataTreeSnapshot(final SchemaContext schemaContext, final StoreMetadataNode rootNode,
+            final ModificationApplyOperation applyOper) {
+        this.schemaContext = Preconditions.checkNotNull(schemaContext);
+        this.rootNode = Preconditions.checkNotNull(rootNode);
+        this.applyOper = Preconditions.checkNotNull(applyOper);
+    }
+
+    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() {
+        return new InMemoryDataTreeModification(this, applyOper);
+    }
+
+    @Override
+    public String toString() {
+        return rootNode.getSubtreeVersion().toString();
+    }
+
+}
\ No newline at end of file
@@ -5,10 +5,9 @@
  * 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 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.StoreTreeNode;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
@@ -38,7 +37,7 @@ import com.google.common.primitives.UnsignedLong;
  *
  *
  */
-public interface ModificationApplyOperation extends StoreTreeNode<ModificationApplyOperation> {
+interface ModificationApplyOperation extends StoreTreeNode<ModificationApplyOperation> {
 
     /**
      *
@@ -80,14 +79,14 @@ public interface ModificationApplyOperation extends StoreTreeNode<ModificationAp
     Optional<ModificationApplyOperation> getChild(PathArgument child);
 
     /**
-    *
-    * Checks if provided node modification could be applied to current metadata node.
-    *
-    * @param modification Modification
-    * @param current Metadata Node to which modification should be applied
-    * @return true if modification is applicable
-    *         false if modification is no applicable
+     *
+     * Checks if provided node modification could be applied to current metadata node.
+     *
+     * @param modification Modification
+     * @param current Metadata Node to which modification should be applied
+     * @return true if modification is applicable
+     *         false if modification is no applicable
      * @throws DataPreconditionFailedException
-    */
+     */
     void checkApplicable(InstanceIdentifier path, NodeModification modification, Optional<StoreMetadataNode> current) throws DataPreconditionFailedException;
 }
@@ -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;
@@ -31,6 +32,7 @@ import com.google.common.base.Predicate;
  * and {@link StoreMetadataNode} which represents original state {@link #getOriginal()}.
  *
  */
+// FIXME: hide this class
 public class NodeModification implements StoreTreeNode<NodeModification>, Identifiable<PathArgument> {
 
     public static final Predicate<NodeModification> IS_TERMINAL_PREDICATE = new Predicate<NodeModification>() {
@@ -67,7 +69,7 @@ public class NodeModification implements StoreTreeNode<NodeModification>, Identi
      *
      * @return
      */
-    public NormalizedNode<?, ?> getWritenValue() {
+    public NormalizedNode<?, ?> getWrittenValue() {
         return value;
     }
 
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,17 +5,15 @@
  * 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 org.opendaylight.controller.md.sal.dom.store.impl.tree.NodeModification;
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.StoreMetadataNode;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 
 import com.google.common.base.Optional;
 import com.google.common.primitives.UnsignedLong;
 
-public class OperationWithModification {
+final class OperationWithModification {
 
     private final NodeModification modification;
 
@@ -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.data;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
@@ -15,10 +15,8 @@ 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.StoreUtils;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.AugmentationIdentifier;
@@ -70,7 +68,7 @@ import com.google.common.cache.LoadingCache;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.primitives.UnsignedLong;
 
-public abstract class SchemaAwareApplyOperation implements ModificationApplyOperation {
+abstract class SchemaAwareApplyOperation implements ModificationApplyOperation {
 
     public static SchemaAwareApplyOperation from(final DataSchemaNode schemaNode) {
         if (schemaNode instanceof ContainerSchemaNode) {
@@ -111,15 +109,17 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
     public static SchemaAwareApplyOperation from(final DataNodeContainer resolvedTree,
             final AugmentationTarget augSchemas, final AugmentationIdentifier identifier) {
         AugmentationSchema augSchema = null;
-        allAugments: for (AugmentationSchema potential : augSchemas.getAvailableAugmentations()) {
-            boolean containsAll = true;
-            for (DataSchemaNode child : potential.getChildNodes()) {
-                if (identifier.getPossibleChildNames().contains(child.getQName())) {
-                    augSchema = potential;
-                    break allAugments;
+
+        allAugments:
+            for (AugmentationSchema potential : augSchemas.getAvailableAugmentations()) {
+                for (DataSchemaNode child : potential.getChildNodes()) {
+                    if (identifier.getPossibleChildNames().contains(child.getQName())) {
+                        augSchema = potential;
+                        break allAugments;
+                    }
                 }
             }
-        }
+
         if (augSchema != null) {
             return new AugmentationModificationStrategy(augSchema, resolvedTree);
         }
@@ -135,11 +135,11 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
     @Override
     public void verifyStructure(final NodeModification modification) throws IllegalArgumentException {
         if (modification.getModificationType() == ModificationType.WRITE) {
-            verifyWritenStructure(modification.getWritenValue());
+            verifyWrittenStructure(modification.getWrittenValue());
         }
     }
 
-    protected abstract void verifyWritenStructure(NormalizedNode<?, ?> writenValue);
+    protected abstract void verifyWrittenStructure(NormalizedNode<?, ?> writtenValue);
 
     @Override
     public void checkApplicable(final InstanceIdentifier path,final NodeModification modification, final Optional<StoreMetadataNode> current) throws DataPreconditionFailedException {
@@ -233,8 +233,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
     protected abstract StoreMetadataNode applySubtreeChange(NodeModification modification,
             StoreMetadataNode currentMeta, UnsignedLong subtreeVersion);
 
-    public static abstract class ValueNodeModificationStrategy<T extends DataSchemaNode> extends
-            SchemaAwareApplyOperation {
+    public static abstract class ValueNodeModificationStrategy<T extends DataSchemaNode> extends SchemaAwareApplyOperation {
 
         private final T schema;
         private final Class<? extends NormalizedNode<?, ?>> nodeClass;
@@ -246,8 +245,8 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
         }
 
         @Override
-        protected void verifyWritenStructure(final NormalizedNode<?, ?> writenValue) {
-            checkArgument(nodeClass.isInstance(writenValue), "Node should must be of type %s", nodeClass);
+        protected void verifyWrittenStructure(final NormalizedNode<?, ?> writtenValue) {
+            checkArgument(nodeClass.isInstance(writtenValue), "Node should must be of type %s", nodeClass);
         }
 
         @Override
@@ -274,7 +273,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
                 final Optional<StoreMetadataNode> currentMeta, final UnsignedLong subtreeVersion) {
             UnsignedLong nodeVersion = subtreeVersion;
             return StoreMetadataNode.builder().setNodeVersion(nodeVersion).setSubtreeVersion(subtreeVersion)
-                    .setData(modification.getWritenValue()).build();
+                    .setData(modification.getWrittenValue()).build();
         }
 
         @Override
@@ -329,13 +328,20 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
 
         @SuppressWarnings("rawtypes")
         @Override
-        protected void verifyWritenStructure(final NormalizedNode<?, ?> writenValue) {
-            checkArgument(nodeClass.isInstance(writenValue), "Node should must be of type %s", nodeClass);
-            checkArgument(writenValue instanceof NormalizedNodeContainer);
-            NormalizedNodeContainer writenCont = (NormalizedNodeContainer) writenValue;
-            for (Object child : writenCont.getValue()) {
+        protected void verifyWrittenStructure(final NormalizedNode<?, ?> writtenValue) {
+            checkArgument(nodeClass.isInstance(writtenValue), "Node should must be of type %s", nodeClass);
+            checkArgument(writtenValue instanceof NormalizedNodeContainer);
+
+            NormalizedNodeContainer container = (NormalizedNodeContainer) writtenValue;
+            for (Object child : container.getValue()) {
                 checkArgument(child instanceof NormalizedNode);
-                NormalizedNode childNode = (NormalizedNode) child;
+
+                /*
+                 * FIXME: fail-fast semantics:
+                 *
+                 * We can validate the data structure here, aborting the commit
+                 * before it ever progresses to being committed.
+                 */
             }
         }
 
@@ -343,7 +349,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
         protected StoreMetadataNode applyWrite(final NodeModification modification,
                 final Optional<StoreMetadataNode> currentMeta, final UnsignedLong subtreeVersion) {
 
-            NormalizedNode<?, ?> newValue = modification.getWritenValue();
+            NormalizedNode<?, ?> newValue = modification.getWrittenValue();
 
             final UnsignedLong nodeVersion;
             if (currentMeta.isPresent()) {
@@ -436,8 +442,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
         protected abstract NormalizedNodeContainerBuilder createBuilder(NormalizedNode<?, ?> original);
     }
 
-    public static abstract class DataNodeContainerModificationStrategy<T extends DataNodeContainer> extends
-            NormalizedNodeContainerModificationStrategy {
+    public static abstract class DataNodeContainerModificationStrategy<T extends DataNodeContainer> extends NormalizedNodeContainerModificationStrategy {
 
         private final T schema;
         private final LoadingCache<PathArgument, ModificationApplyOperation> childCache = CacheBuilder.newBuilder()
@@ -487,8 +492,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
 
     }
 
-    public static class ContainerModificationStrategy extends
-            DataNodeContainerModificationStrategy<ContainerSchemaNode> {
+    public static class ContainerModificationStrategy extends DataNodeContainerModificationStrategy<ContainerSchemaNode> {
 
         public ContainerModificationStrategy(final ContainerSchemaNode schemaNode) {
             super(schemaNode, ContainerNode.class);
@@ -502,8 +506,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
         }
     }
 
-    public static class UnkeyedListItemModificationStrategy extends
-            DataNodeContainerModificationStrategy<ListSchemaNode> {
+    public static class UnkeyedListItemModificationStrategy extends DataNodeContainerModificationStrategy<ListSchemaNode> {
 
         public UnkeyedListItemModificationStrategy(final ListSchemaNode schemaNode) {
             super(schemaNode, UnkeyedListEntryNode.class);
@@ -517,8 +520,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
         }
     }
 
-    public static class AugmentationModificationStrategy extends
-            DataNodeContainerModificationStrategy<AugmentationSchema> {
+    public static class AugmentationModificationStrategy extends DataNodeContainerModificationStrategy<AugmentationSchema> {
 
         protected AugmentationModificationStrategy(final AugmentationSchema schema, final DataNodeContainer resolved) {
             super(createAugmentProxy(schema,resolved), AugmentationNode.class);
@@ -534,12 +536,10 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
 
     public static class ChoiceModificationStrategy extends NormalizedNodeContainerModificationStrategy {
 
-        private final ChoiceNode schema;
         private final Map<PathArgument, ModificationApplyOperation> childNodes;
 
         public ChoiceModificationStrategy(final ChoiceNode schemaNode) {
             super(org.opendaylight.yangtools.yang.data.api.schema.ChoiceNode.class);
-            this.schema = schemaNode;
             ImmutableMap.Builder<PathArgument, ModificationApplyOperation> child = ImmutableMap.builder();
 
             for (ChoiceCaseNode caze : schemaNode.getCases()) {
@@ -653,7 +653,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
         @Override
         protected StoreMetadataNode applyWrite(final NodeModification modification,
                 final Optional<StoreMetadataNode> currentMeta, final UnsignedLong subtreeVersion) {
-            return StoreMetadataNode.createRecursively(modification.getWritenValue(), subtreeVersion);
+            return StoreMetadataNode.createRecursively(modification.getWrittenValue(), subtreeVersion);
         }
 
         @Override
@@ -665,7 +665,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
         }
 
         @Override
-        protected void verifyWritenStructure(final NormalizedNode<?, ?> writenValue) {
+        protected void verifyWrittenStructure(final NormalizedNode<?, ?> writtenValue) {
 
         }
 
@@ -737,10 +737,6 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper
         }
     }
 
-    public void verifyIdentifier(final PathArgument identifier) {
-
-    }
-
     public static AugmentationSchema createAugmentProxy(final AugmentationSchema schema, final DataNodeContainer resolved) {
         Set<DataSchemaNode> realChildSchemas = new HashSet<>();
         for(DataSchemaNode augChild : schema.getChildNodes()) {
@@ -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;
@@ -23,6 +24,7 @@ import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.UnsignedLong;
 
+// FIXME: this should not be public
 public class StoreMetadataNode implements Immutable, Identifiable<PathArgument>, StoreTreeNode<StoreMetadataNode> {
 
     private final UnsignedLong nodeVersion;
@@ -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;
@@ -20,7 +20,7 @@ import com.google.common.primitives.UnsignedLong;
  *
  */
 @SuppressWarnings("rawtypes")
-public class StoreNodeCompositeBuilder {
+class StoreNodeCompositeBuilder {
 
     private final StoreMetadataNode.Builder metadata;
 
@@ -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,9 @@ 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.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;
@@ -89,14 +98,16 @@ public class ModificationMetadataTreeTest {
                     .withChild(mapEntry(INNER_LIST_QNAME, NAME_QNAME, TWO_ONE_NAME)) //
                     .withChild(mapEntry(INNER_LIST_QNAME, NAME_QNAME, TWO_TWO_NAME)) //
                     .build()) //
-            .build();
+                    .build();
 
     private SchemaContext schemaContext;
+    private ModificationApplyOperation applyOper;
 
     @Before
     public void prepare() {
         schemaContext = TestModel.createTestContext();
         assertNotNull("Schema context must not be null.", schemaContext);
+        applyOper = SchemaAwareApplyOperation.from(schemaContext);
     }
 
     /**
@@ -131,16 +142,16 @@ public class ModificationMetadataTreeTest {
                 .withNodeIdentifier(new NodeIdentifier(TEST_QNAME))
                 .withChild(
                         mapNodeBuilder(OUTER_LIST_QNAME)
-                                .withChild(mapEntry(OUTER_LIST_QNAME, ID_QNAME, ONE_ID))
-                                .withChild(BAR_NODE).build()).build();
+                        .withChild(mapEntry(OUTER_LIST_QNAME, ID_QNAME, ONE_ID))
+                        .withChild(BAR_NODE).build()).build();
     }
 
     @Test
     public void basicReadWrites() {
-        MutableDataTree modificationTree = MutableDataTree.from(new DataTree.Snapshot(schemaContext,
-                StoreMetadataNode.createRecursively(createDocumentOne(), UnsignedLong.valueOf(5))),
+        DataTreeModification modificationTree = new InMemoryDataTreeModification(new InMemoryDataTreeSnapshot(schemaContext,
+                StoreMetadataNode.createRecursively(createDocumentOne(), UnsignedLong.valueOf(5)), applyOper),
                 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,22 +160,23 @@ 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());
     }
 
 
-    public MutableDataTree createEmptyModificationTree() {
+    public DataTreeModification createEmptyModificationTree() {
         /**
          * Creates empty Snapshot with associated schema context.
          */
-        DataTree t = DataTree.create(schemaContext);
+        DataTree t = InMemoryDataTreeFactory.getInstance().create();
+        t.setSchemaContext(schemaContext);
 
         /**
          *
@@ -172,15 +184,13 @@ public class ModificationMetadataTreeTest {
          * context.
          *
          */
-        MutableDataTree modificationTree = MutableDataTree.from(t.takeSnapshot(), new SchemaAwareApplyOperationRoot(
-                schemaContext));
-        return modificationTree;
+        return t.takeSnapshot().newModification();
     }
 
     @Test
     public void createFromEmptyState() {
 
-        MutableDataTree modificationTree = createEmptyModificationTree();
+        DataTreeModification modificationTree = createEmptyModificationTree();
         /**
          * Writes empty container node to /test
          *
@@ -195,14 +205,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);
 
         /**
@@ -217,23 +227,23 @@ public class ModificationMetadataTreeTest {
 
     @Test
     public void writeSubtreeReadChildren() {
-        MutableDataTree modificationTree = createEmptyModificationTree();
+        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
     public void writeSubtreeDeleteChildren() {
-        MutableDataTree modificationTree = createEmptyModificationTree();
+        DataTreeModification modificationTree = createEmptyModificationTree();
         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());
 
     }
@@ -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.data;
 
 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;