From: Tony Tkacik Date: Tue, 20 May 2014 14:57:56 +0000 (+0000) Subject: Merge "Bug 953 - Change the BindingIndependentConnector to handle RpcInvocationStrate... X-Git-Tag: autorelease-tag-v20140601202136_82eb3f9~50 X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=commitdiff_plain;h=c3c49eaef5b1435b871f97eb060f8abdba5a6671;hp=d449631e9b99aace639985286b73357577481cab Merge "Bug 953 - Change the BindingIndependentConnector to handle RpcInvocationStrategies consistently." --- 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 index 6baf7647bd..0000000000 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataPreconditionFailedException.java +++ /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 index 3124199006..0000000000 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/DataTree.java +++ /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> 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(); - } - } -} diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java index 7d2ff30b1f..9e11fc70fc 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java @@ -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, 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, 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, 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, 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, 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, Sch return store.submit(this); } - protected MutableDataTree getMutatedView() { + protected DataTreeModification getMutatedView() { return mutableTree; } @@ -306,18 +272,18 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, 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>> 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, Sch private class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort { private final SnapshotBackedWriteTransaction transaction; - private final NodeModification modification; + private final DataTreeModification modification; - private DataTree.Snapshot storeSnapshot; - private Optional 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 canCommit() { - final DataTree.Snapshot snapshotCapture = dataTree.takeSnapshot(); - final ModificationApplyOperation snapshotOperation = operationTree; - return executor.submit(new Callable() { - @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 preCommit() { - storeSnapshot = dataTree.takeSnapshot(); - if (modification.getModificationType() == ModificationType.UNMODIFIED) { - return Futures.immediateFuture(null); - } return executor.submit(new Callable() { - @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, Sch @Override public ListenableFuture abort() { - storeSnapshot = null; - proposedSubtree = null; + if (candidate != null) { + candidate.close(); + candidate = null; + } + return Futures. immediateFuture(null); } @Override public ListenableFuture 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. immediateFuture(null); } - - } - - private static final class AlwaysFailOperation implements ModificationApplyOperation { - - @Override - public Optional apply(final NodeModification modification, - final Optional storeMeta, final UnsignedLong subtreeVersion) { - throw new IllegalStateException("Schema Context is not available."); - } - - @Override - public void checkApplicable(final InstanceIdentifier path,final NodeModification modification, final Optional storeMetadata) { - throw new IllegalStateException("Schema Context is not available."); - } - - @Override - public Optional 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."); - } - } } diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ResolveDataChangeEventsTask.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ResolveDataChangeEventsTask.java index 44d50166af..db9bb0fef2 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ResolveDataChangeEventsTask.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ResolveDataChangeEventsTask.java @@ -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; * * */ -public class ResolveDataChangeEventsTask implements Callable> { +final class ResolveDataChangeEventsTask implements Callable> { 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 beforeRoot; - private Optional afterRoot; private final Multimap 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 getBeforeRoot() { - return beforeRoot; - } - - protected ResolveDataChangeEventsTask setBeforeRoot(final Optional beforeRoot) { - this.beforeRoot = beforeRoot; - return this; - } - - protected Optional getAfterRoot() { - return afterRoot; - } - - protected ResolveDataChangeEventsTask setAfterRoot(final Optional 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 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 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 index 0000000000..a94acc5658 --- /dev/null +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/DataTreeSnapshot.java @@ -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> 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(); +} diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/StoreUtils.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreUtils.java similarity index 98% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/StoreUtils.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreUtils.java index e1da9a7381..7e783f927d 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/StoreUtils.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreUtils.java @@ -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 index 0000000000..b2faf79565 --- /dev/null +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/AbstractDataTreeCandidate.java @@ -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 index 0000000000..4e3aa49113 --- /dev/null +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/AlwaysFailOperation.java @@ -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 apply(final NodeModification modification, + final Optional storeMeta, final UnsignedLong subtreeVersion) { + throw new IllegalStateException("Schema Context is not available."); + } + + @Override + public void checkApplicable(final InstanceIdentifier path,final NodeModification modification, final Optional storeMetadata) { + throw new IllegalStateException("Schema Context is not available."); + } + + @Override + public Optional 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 index 0000000000..5a300a071d --- /dev/null +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTree.java @@ -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 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 index 0000000000..93719b7f53 --- /dev/null +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeCandidate.java @@ -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 index 0000000000..7614611ab2 --- /dev/null +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeFactory.java @@ -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; + } +} diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/MutableDataTree.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeModification.java similarity index 69% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/MutableDataTree.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeModification.java index 1002cd54b5..df3ef8b7e1 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/MutableDataTree.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeModification.java @@ -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 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> read(final InstanceIdentifier path) { - Entry modification = TreeNodeUtils.findClosestsOrFirstMatch(rootModification, path, NodeModification.IS_TERMINAL_PREDICATE); + @Override + public Optional> readNode(final InstanceIdentifier path) { + Entry modification = TreeNodeUtils.findClosestsOrFirstMatch(rootNode, path, NodeModification.IS_TERMINAL_PREDICATE); Optional 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 index 0000000000..ce2d8c9bd4 --- /dev/null +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/InMemoryDataTreeSnapshot.java @@ -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> 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 diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationApplyOperation.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationApplyOperation.java similarity index 83% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationApplyOperation.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationApplyOperation.java index 361be6800c..5b4cd565e5 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationApplyOperation.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationApplyOperation.java @@ -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 { +interface ModificationApplyOperation extends StoreTreeNode { /** * @@ -80,14 +79,14 @@ public interface ModificationApplyOperation extends StoreTreeNode 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 current) throws DataPreconditionFailedException; } diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/ModificationType.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationType.java similarity index 91% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/ModificationType.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationType.java index b16e907120..9d2e965ff7 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/ModificationType.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationType.java @@ -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 { diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/NodeModification.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/NodeModification.java similarity index 96% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/NodeModification.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/NodeModification.java index 4f650c1711..18179afd50 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/NodeModification.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/NodeModification.java @@ -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, Identifiable { public static final Predicate IS_TERMINAL_PREDICATE = new Predicate() { @@ -67,7 +69,7 @@ public class NodeModification implements StoreTreeNode, 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 index 0000000000..1782da2835 --- /dev/null +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/NoopDataTreeCandidate.java @@ -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; + } +} diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/OperationWithModification.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/OperationWithModification.java similarity index 89% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/OperationWithModification.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/OperationWithModification.java index 780291e70f..fda8407a95 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/OperationWithModification.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/OperationWithModification.java @@ -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; diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperation.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/SchemaAwareApplyOperation.java similarity index 93% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperation.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/SchemaAwareApplyOperation.java index afe9653394..02244d9f98 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperation.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/SchemaAwareApplyOperation.java @@ -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 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 extends - SchemaAwareApplyOperation { + public static abstract class ValueNodeModificationStrategy extends SchemaAwareApplyOperation { private final T schema; private final Class> 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 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 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 extends - NormalizedNodeContainerModificationStrategy { + public static abstract class DataNodeContainerModificationStrategy extends NormalizedNodeContainerModificationStrategy { private final T schema; private final LoadingCache childCache = CacheBuilder.newBuilder() @@ -487,8 +492,7 @@ public abstract class SchemaAwareApplyOperation implements ModificationApplyOper } - public static class ContainerModificationStrategy extends - DataNodeContainerModificationStrategy { + public static class ContainerModificationStrategy extends DataNodeContainerModificationStrategy { 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 { + public static class UnkeyedListItemModificationStrategy extends DataNodeContainerModificationStrategy { 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 { + public static class AugmentationModificationStrategy extends DataNodeContainerModificationStrategy { 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 childNodes; public ChoiceModificationStrategy(final ChoiceNode schemaNode) { super(org.opendaylight.yangtools.yang.data.api.schema.ChoiceNode.class); - this.schema = schemaNode; ImmutableMap.Builder 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 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 realChildSchemas = new HashSet<>(); for(DataSchemaNode augChild : schema.getChildNodes()) { diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreMetadataNode.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/StoreMetadataNode.java similarity index 97% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreMetadataNode.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/StoreMetadataNode.java index b8ad7368b5..8addb89bd1 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreMetadataNode.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/StoreMetadataNode.java @@ -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, StoreTreeNode { private final UnsignedLong nodeVersion; diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreNodeCompositeBuilder.java b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/StoreNodeCompositeBuilder.java similarity index 96% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreNodeCompositeBuilder.java rename to opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/StoreNodeCompositeBuilder.java index a66a1d5b1c..6bce4fff0c 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/StoreNodeCompositeBuilder.java +++ b/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/StoreNodeCompositeBuilder.java @@ -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; diff --git a/opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationMetadataTreeTest.java b/opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationMetadataTreeTest.java similarity index 81% rename from opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationMetadataTreeTest.java rename to opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationMetadataTreeTest.java index 0445c47c6b..8940e55d32 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/ModificationMetadataTreeTest.java +++ b/opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/ModificationMetadataTreeTest.java @@ -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> originalBarNode = modificationTree.read(OUTER_LIST_2_PATH); + Optional> 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> barTwoCModified = modificationTree.read(TWO_TWO_VALUE_PATH); + Optional> 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> barTwoCAfterDelete = modificationTree.read(TWO_TWO_VALUE_PATH); + Optional> 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> potentialOuterList = modificationTree.read(OUTER_LIST_PATH); + Optional> potentialOuterList = modificationTree.readNode(OUTER_LIST_PATH); assertTrue(potentialOuterList.isPresent()); /** * Reads container node from /test and verifies that it contains test * node */ - Optional> potentialTest = modificationTree.read(TEST_PATH); + Optional> 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> potential = modificationTree.read(TWO_TWO_PATH); - MapEntryNode node = assertPresentAndType(potential, MapEntryNode.class); + Optional> 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> potentialBeforeDelete = modificationTree.read(TWO_TWO_PATH); - MapEntryNode node = assertPresentAndType(potentialBeforeDelete, MapEntryNode.class); + Optional> potentialBeforeDelete = modificationTree.readNode(TWO_TWO_PATH); + assertPresentAndType(potentialBeforeDelete, MapEntryNode.class); modificationTree.delete(TWO_TWO_PATH); - Optional> potentialAfterDelete = modificationTree.read(TWO_TWO_PATH); + Optional> potentialAfterDelete = modificationTree.readNode(TWO_TWO_PATH); assertFalse(potentialAfterDelete.isPresent()); } diff --git a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperationRoot.java b/opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/SchemaAwareApplyOperationRoot.java similarity index 95% rename from opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperationRoot.java rename to opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/SchemaAwareApplyOperationRoot.java index 8a539ff36e..f2cc533207 100644 --- a/opendaylight/md-sal/sal-dom-broker/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/SchemaAwareApplyOperationRoot.java +++ b/opendaylight/md-sal/sal-dom-broker/src/test/java/org/opendaylight/controller/md/sal/dom/store/impl/tree/data/SchemaAwareApplyOperationRoot.java @@ -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;