Merge "Removed `which` dependency, now using proper shell builtin."
[controller.git] / opendaylight / md-sal / sal-dom-broker / src / main / java / org / opendaylight / controller / md / sal / dom / store / impl / InMemoryDOMDataStore.java
index 7d647af53907242e9c06dec14dcfa81e05c2fad6..10b838a2c6122a2d8e629f7ce02f3698fe90703c 100644 (file)
@@ -7,62 +7,64 @@
  */
 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 java.util.Collections;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicLong;
 
+import javax.annotation.concurrent.GuardedBy;
+
 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.SnapshotBackedWriteTransaction.TransactionReadyPrototype;
 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.ModificationApplyOperation;
 import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.InMemoryDataTreeFactory;
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.NodeModification;
-import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.StoreMetadataNode;
 import org.opendaylight.controller.sal.core.spi.data.DOMStore;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransaction;
+import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
 import org.opendaylight.yangtools.concepts.AbstractListenerRegistration;
 import org.opendaylight.yangtools.concepts.Identifiable;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.opendaylight.yangtools.yang.model.api.SchemaContextListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Objects;
-import com.google.common.base.Objects.ToStringHelper;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.primitives.UnsignedLong;
+import com.google.common.util.concurrent.FutureCallback;
 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 {
-
+/**
+ * In-memory DOM Data Store
+ *
+ * Implementation of {@link DOMStore} which uses {@link DataTree} and other
+ * classes such as {@link SnapshotBackedWriteTransaction}.
+ * {@link SnapshotBackedReadTransaction} and {@link ResolveDataChangeEventsTask}
+ * to implement {@link DOMStore} contract.
+ *
+ */
+public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, SchemaContextListener,
+        TransactionReadyPrototype {
     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 = InMemoryDataTreeFactory.getInstance().create();
-    private ModificationApplyOperation operationTree = new AlwaysFailOperation();
 
     public InMemoryDOMDataStore(final String name, final ListeningExecutorService executor) {
         this.name = Preconditions.checkNotNull(name);
@@ -81,25 +83,22 @@ 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 DOMStoreTransactionChain createTransactionChain() {
+        return new DOMStoreTransactionChainImpl();
     }
 
     @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
@@ -141,7 +140,8 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         };
     }
 
-    private synchronized DOMStoreThreePhaseCommitCohort submit(final SnapshotBackedWriteTransaction writeTx) {
+    @Override
+    public synchronized DOMStoreThreePhaseCommitCohort ready(final SnapshotBackedWriteTransaction writeTx) {
         LOG.debug("Tx: {} is submitted. Modifications: {}", writeTx.getIdentifier(), writeTx.getMutatedView());
         return new ThreePhaseCommitImpl(writeTx);
     }
@@ -150,162 +150,139 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
         return name + "-" + txCounter.getAndIncrement();
     }
 
-    private static abstract class AbstractDOMStoreTransaction implements DOMStoreTransaction {
-        private final Object identifier;
+    private class DOMStoreTransactionChainImpl implements DOMStoreTransactionChain, TransactionReadyPrototype {
+
+        @GuardedBy("this")
+        private SnapshotBackedWriteTransaction latestOutstandingTx;
 
-        protected AbstractDOMStoreTransaction(final Object identifier) {
-            this.identifier = identifier;
+        private boolean chainFailed = false;
+
+        private void checkFailed() {
+            Preconditions.checkState(!chainFailed, "Transaction chain is failed.");
         }
 
         @Override
-        public final Object getIdentifier() {
-            return identifier;
+        public synchronized DOMStoreReadTransaction newReadOnlyTransaction() {
+            final DataTreeSnapshot snapshot;
+            checkFailed();
+            if (latestOutstandingTx != null) {
+                checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
+                snapshot = latestOutstandingTx.getMutatedView();
+            } else {
+                snapshot = dataTree.takeSnapshot();
+            }
+            return new SnapshotBackedReadTransaction(nextIdentifier(), snapshot);
         }
 
         @Override
-        public final String toString() {
-            return addToStringAttributes(Objects.toStringHelper(this)).toString();
+        public synchronized DOMStoreReadWriteTransaction newReadWriteTransaction() {
+            final DataTreeSnapshot snapshot;
+            checkFailed();
+            if (latestOutstandingTx != null) {
+                checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
+                snapshot = latestOutstandingTx.getMutatedView();
+            } else {
+                snapshot = dataTree.takeSnapshot();
+            }
+            final SnapshotBackedReadWriteTransaction ret = new SnapshotBackedReadWriteTransaction(nextIdentifier(),
+                    snapshot, this);
+            latestOutstandingTx = ret;
+            return ret;
         }
 
-        /**
-         * Add class-specific toString attributes.
-         *
-         * @param toStringHelper
-         *            ToStringHelper instance
-         * @return ToStringHelper instance which was passed in
-         */
-        protected ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
-            return toStringHelper.add("id", identifier);
+        @Override
+        public synchronized DOMStoreWriteTransaction newWriteOnlyTransaction() {
+            final DataTreeSnapshot snapshot;
+            checkFailed();
+            if (latestOutstandingTx != null) {
+                checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
+                snapshot = latestOutstandingTx.getMutatedView();
+            } else {
+                snapshot = dataTree.takeSnapshot();
+            }
+            final SnapshotBackedWriteTransaction ret = new SnapshotBackedWriteTransaction(nextIdentifier(), snapshot,
+                    this);
+            latestOutstandingTx = ret;
+            return ret;
         }
-    }
-
-    private static final class SnapshotBackedReadTransaction extends AbstractDOMStoreTransaction implements
-            DOMStoreReadTransaction {
-        private DataTreeSnapshot stableSnapshot;
 
-        public SnapshotBackedReadTransaction(final Object identifier, final DataTreeSnapshot snapshot) {
-            super(identifier);
-            this.stableSnapshot = Preconditions.checkNotNull(snapshot);
-            LOG.debug("ReadOnly Tx: {} allocated with snapshot {}", identifier, snapshot);
+        @Override
+        public DOMStoreThreePhaseCommitCohort ready(final SnapshotBackedWriteTransaction tx) {
+            DOMStoreThreePhaseCommitCohort storeCohort = InMemoryDOMDataStore.this.ready(tx);
+            return new ChainedTransactionCommitImpl(tx, storeCohort, this);
         }
 
         @Override
         public void close() {
-            LOG.debug("Store transaction: {} : Closed", getIdentifier());
-            stableSnapshot = null;
-        }
 
-        @Override
-        public ListenableFuture<Optional<NormalizedNode<?, ?>>> read(final InstanceIdentifier path) {
-            checkNotNull(path, "Path must not be null.");
-            checkState(stableSnapshot != null, "Transaction is closed");
-            return Futures.immediateFuture(stableSnapshot.readNode(path));
         }
-    }
 
-    private static class SnapshotBackedWriteTransaction extends AbstractDOMStoreTransaction implements
-            DOMStoreWriteTransaction {
-        private DataTreeModification mutableTree;
-        private InMemoryDOMDataStore store;
-        private boolean ready = false;
-
-        public SnapshotBackedWriteTransaction(final Object identifier, final DataTreeSnapshot snapshot,
-                final InMemoryDOMDataStore store, final ModificationApplyOperation applyOper) {
-            super(identifier);
-            mutableTree = snapshot.newModification(applyOper);
-            this.store = store;
-            LOG.debug("Write Tx: {} allocated with snapshot {}", identifier, snapshot);
-        }
+        protected synchronized void onTransactionFailed(final SnapshotBackedWriteTransaction transaction,
+                final Throwable t) {
+            chainFailed = true;
 
-        @Override
-        public void close() {
-            LOG.debug("Store transaction: {} : Closed", getIdentifier());
-            this.mutableTree = null;
-            this.store = null;
         }
 
-        @Override
-        public void write(final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
-            checkNotReady();
-            try {
-                LOG.trace("Tx: {} Write: {}:{}", getIdentifier(), path, data);
-                mutableTree.write(path, data);
-                // FIXME: Add checked exception
-            } catch (Exception e) {
-                LOG.error("Tx: {}, failed to write {}:{} in {}", getIdentifier(), path, data, mutableTree, e);
+        public synchronized void onTransactionCommited(final SnapshotBackedWriteTransaction transaction) {
+            // If commited transaction is latestOutstandingTx we clear
+            // latestOutstandingTx
+            // field in order to base new transactions on Datastore Data Tree
+            // directly.
+            if (transaction.equals(latestOutstandingTx)) {
+                latestOutstandingTx = null;
             }
         }
 
-        @Override
-        public void merge(final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
-            checkNotReady();
-            try {
-                LOG.trace("Tx: {} Merge: {}:{}", getIdentifier(), path, data);
-                mutableTree.merge(path, data);
-                // FIXME: Add checked exception
-            } catch (Exception e) {
-                LOG.error("Tx: {}, failed to write {}:{} in {}", getIdentifier(), path, data, mutableTree, e);
-            }
-        }
+    }
 
-        @Override
-        public void delete(final InstanceIdentifier path) {
-            checkNotReady();
-            try {
-                LOG.trace("Tx: {} Delete: {}", getIdentifier(), path);
-                mutableTree.delete(path);
-                // FIXME: Add checked exception
-            } catch (Exception e) {
-                LOG.error("Tx: {}, failed to delete {} in {}", getIdentifier(), path, mutableTree, e);
-            }
-        }
+    private static class ChainedTransactionCommitImpl implements DOMStoreThreePhaseCommitCohort {
 
-        protected final boolean isReady() {
-            return ready;
-        }
+        private final SnapshotBackedWriteTransaction transaction;
+        private final DOMStoreThreePhaseCommitCohort delegate;
+
+        private final DOMStoreTransactionChainImpl txChain;
 
-        protected final void checkNotReady() {
-            checkState(!ready, "Transaction %s is ready. No further modifications allowed.", getIdentifier());
+        protected ChainedTransactionCommitImpl(final SnapshotBackedWriteTransaction transaction,
+                final DOMStoreThreePhaseCommitCohort delegate, final DOMStoreTransactionChainImpl txChain) {
+            super();
+            this.transaction = transaction;
+            this.delegate = delegate;
+            this.txChain = txChain;
         }
 
         @Override
-        public synchronized DOMStoreThreePhaseCommitCohort ready() {
-            checkState(!ready, "Transaction %s is already ready.", getIdentifier());
-            ready = true;
-
-            LOG.debug("Store transaction: {} : Ready", getIdentifier());
-            mutableTree.seal();
-            return store.submit(this);
+        public ListenableFuture<Boolean> canCommit() {
+            return delegate.canCommit();
         }
 
-        protected DataTreeModification getMutatedView() {
-            return mutableTree;
+        @Override
+        public ListenableFuture<Void> preCommit() {
+            return delegate.preCommit();
         }
 
         @Override
-        protected ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
-            return toStringHelper.add("ready", isReady());
+        public ListenableFuture<Void> abort() {
+            return delegate.abort();
         }
-    }
 
-    private static class SnapshotBackedReadWriteTransaction extends SnapshotBackedWriteTransaction implements
-            DOMStoreReadWriteTransaction {
+        @Override
+        public ListenableFuture<Void> commit() {
+            ListenableFuture<Void> commitFuture = delegate.commit();
+            Futures.addCallback(commitFuture, new FutureCallback<Void>() {
+                @Override
+                public void onFailure(final Throwable t) {
+                    txChain.onTransactionFailed(transaction, t);
+                }
 
-        protected SnapshotBackedReadWriteTransaction(final Object identifier, final DataTreeSnapshot snapshot,
-                final InMemoryDOMDataStore store, final ModificationApplyOperation applyOper) {
-            super(identifier, snapshot, store, applyOper);
-        }
+                @Override
+                public void onSuccess(final Void result) {
+                    txChain.onTransactionCommited(transaction);
+                }
 
-        @Override
-        public ListenableFuture<Optional<NormalizedNode<?, ?>>> read(final InstanceIdentifier path) {
-            LOG.trace("Tx: {} Read: {}", getIdentifier(), path);
-            try {
-                return Futures.immediateFuture(getMutatedView().readNode(path));
-            } catch (Exception e) {
-                LOG.error("Tx: {} Failed Read of {}", getIdentifier(), path, e);
-                throw e;
-            }
+            });
+            return commitFuture;
         }
+
     }
 
     private class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort {
@@ -327,11 +304,12 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
                 @Override
                 public Boolean call() {
                     try {
-                       dataTree.validate(modification);
+                        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);
+                        LOG.warn("Store Tx: {} Data Precondition failed for {}.", transaction.getIdentifier(),
+                                e.getPath(), e);
                         return false;
                     }
                 }
@@ -343,16 +321,8 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
             return executor.submit(new Callable<Void>() {
                 @Override
                 public Void call() {
-                       candidate = dataTree.prepare(modification);
-
+                    candidate = dataTree.prepare(modification);
                     listenerResolver = ResolveDataChangeEventsTask.create(candidate, listenerTree);
-
-//                            .setRootPath(PUBLIC_ROOT_PATH) //
-//                            .setBeforeRoot(Optional.of(metadataTree)) //
-//                            .setAfterRoot(proposedSubtree) //
-//                            .setModificationRoot(modification.getRootModification()) //
-//                            .setListenerRoot(listenerTree);
-
                     return null;
                 }
             });
@@ -360,12 +330,8 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
 
         @Override
         public ListenableFuture<Void> abort() {
-               if (candidate != null) {
-                       candidate.close();
-                       candidate = null;
-               }
-
-            return Futures.<Void> immediateFuture(null);
+            candidate = null;
+            return Futures.immediateFuture(null);
         }
 
         @Override
@@ -385,32 +351,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, Sch
                 }
             }
 
-            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.");
+            return Futures.immediateFuture(null);
         }
-
-        @Override
-        public void verifyStructure(final NodeModification modification) throws IllegalArgumentException {
-            throw new IllegalStateException("Schema Context is not available.");
-        }
-
     }
 }