Bug 4105: Implement RegisterCandidate in EntityOwnershipShard
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / AbstractShardTest.java
index 378bc717f4da3b22f2516a2c303c43abc9144fab..97c9e923a9b656eb067242f1c27bb8a121dd13e8 100644 (file)
@@ -16,11 +16,11 @@ import static org.mockito.Mockito.mock;
 import akka.actor.ActorRef;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
+import akka.dispatch.Dispatchers;
 import akka.japi.Creator;
 import akka.testkit.TestActorRef;
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
-import com.google.common.util.concurrent.CheckedFuture;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
 import java.util.Collections;
@@ -41,16 +41,17 @@ import org.opendaylight.controller.cluster.datastore.modification.WriteModificat
 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
-import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
-import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeSnapshot;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 
 /**
@@ -91,14 +92,15 @@ public abstract class AbstractShardTest extends AbstractActorTest{
                 newDatastoreContext(), SCHEMA_CONTEXT);
     }
 
-    protected void testRecovery(Set<Integer> listEntryKeys) throws Exception {
+    protected void testRecovery(final Set<Integer> listEntryKeys) throws Exception {
         // Create the actor and wait for recovery complete.
 
-        int nListEntries = listEntryKeys.size();
+        final int nListEntries = listEntryKeys.size();
 
         final CountDownLatch recoveryComplete = new CountDownLatch(1);
 
         @SuppressWarnings("serial")
+        final
         Creator<Shard> creator = new Creator<Shard>() {
             @Override
             public Shard create() throws Exception {
@@ -116,25 +118,25 @@ public abstract class AbstractShardTest extends AbstractActorTest{
             }
         };
 
-        TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                Props.create(new DelegatingShardCreator(creator)), "testRecovery");
+        final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "testRecovery");
 
         assertEquals("Recovery complete", true, recoveryComplete.await(5, TimeUnit.SECONDS));
 
         // Verify data in the data store.
 
-        NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
+        final NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
         assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
         assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
                 outerList.getValue() instanceof Iterable);
-        for(Object entry: (Iterable<?>) outerList.getValue()) {
+        for(final Object entry: (Iterable<?>) outerList.getValue()) {
             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
                     entry instanceof MapEntryNode);
-            MapEntryNode mapEntry = (MapEntryNode)entry;
-            Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
+            final MapEntryNode mapEntry = (MapEntryNode)entry;
+            final Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
                     mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
             assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
-            Object value = idLeaf.get().getValue();
+            final Object value = idLeaf.get().getValue();
             assertTrue("Unexpected value for leaf "+ TestModel.ID_QNAME.getLocalName() + ": " + value,
                     listEntryKeys.remove(value));
         }
@@ -154,7 +156,7 @@ public abstract class AbstractShardTest extends AbstractActorTest{
         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
     }
 
-    protected void verifyLastApplied(TestActorRef<Shard> shard, long expectedValue) {
+    protected void verifyLastApplied(final TestActorRef<Shard> shard, final long expectedValue) {
         long lastApplied = -1;
         for(int i = 0; i < 20 * 5; i++) {
             lastApplied = shard.underlyingActor().getShardMBean().getLastApplied();
@@ -167,49 +169,35 @@ public abstract class AbstractShardTest extends AbstractActorTest{
         Assert.fail(String.format("Expected last applied: %d, Actual: %d", expectedValue, lastApplied));
     }
 
-    protected NormalizedNode<?, ?> readStore(final InMemoryDOMDataStore store) throws ReadFailedException {
-        DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read =
-            transaction.read(YangInstanceIdentifier.builder().build());
-
-        Optional<NormalizedNode<?, ?>> optional = read.checkedGet();
-
-        NormalizedNode<?, ?> normalizedNode = optional.get();
-
-        transaction.close();
-
-        return normalizedNode;
-    }
-
-    protected DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(final String cohortName,
-            final InMemoryDOMDataStore dataStore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
+    protected ShardDataTreeCohort setupMockWriteTransaction(final String cohortName,
+            final ShardDataTree dataStore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
             final MutableCompositeModification modification) {
         return setupMockWriteTransaction(cohortName, dataStore, path, data, modification, null);
     }
 
-    protected DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(final String cohortName,
-            final InMemoryDOMDataStore dataStore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
+    protected ShardDataTreeCohort setupMockWriteTransaction(final String cohortName,
+            final ShardDataTree dataStore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
             final MutableCompositeModification modification,
-            final Function<DOMStoreThreePhaseCommitCohort,ListenableFuture<Void>> preCommit) {
+            final Function<ShardDataTreeCohort, ListenableFuture<Void>> preCommit) {
 
-        DOMStoreWriteTransaction tx = dataStore.newWriteOnlyTransaction();
-        tx.write(path, data);
-        DOMStoreThreePhaseCommitCohort cohort = createDelegatingMockCohort(cohortName, tx.ready(), preCommit);
+        final ReadWriteShardDataTreeTransaction tx = dataStore.newReadWriteTransaction("setup-mock-" + cohortName, null);
+        tx.getSnapshot().write(path, data);
+        final ShardDataTreeCohort cohort = createDelegatingMockCohort(cohortName, dataStore.finishTransaction(tx), preCommit);
 
         modification.addModification(new WriteModification(path, data));
 
         return cohort;
     }
 
-    protected DOMStoreThreePhaseCommitCohort createDelegatingMockCohort(final String cohortName,
-            final DOMStoreThreePhaseCommitCohort actual) {
+    protected ShardDataTreeCohort createDelegatingMockCohort(final String cohortName,
+            final ShardDataTreeCohort actual) {
         return createDelegatingMockCohort(cohortName, actual, null);
     }
 
-    protected DOMStoreThreePhaseCommitCohort createDelegatingMockCohort(final String cohortName,
-            final DOMStoreThreePhaseCommitCohort actual,
-            final Function<DOMStoreThreePhaseCommitCohort,ListenableFuture<Void>> preCommit) {
-        DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, cohortName);
+    protected ShardDataTreeCohort createDelegatingMockCohort(final String cohortName,
+            final ShardDataTreeCohort actual,
+            final Function<ShardDataTreeCohort, ListenableFuture<Void>> preCommit) {
+        final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, cohortName);
 
         doAnswer(new Answer<ListenableFuture<Boolean>>() {
             @Override
@@ -221,7 +209,11 @@ public abstract class AbstractShardTest extends AbstractActorTest{
         doAnswer(new Answer<ListenableFuture<Void>>() {
             @Override
             public ListenableFuture<Void> answer(final InvocationOnMock invocation) throws Throwable {
-                return actual.preCommit();
+                if(preCommit != null) {
+                    return preCommit.apply(actual);
+                } else {
+                    return actual.preCommit();
+                }
             }
         }).when(cohort).preCommit();
 
@@ -239,43 +231,55 @@ public abstract class AbstractShardTest extends AbstractActorTest{
             }
         }).when(cohort).abort();
 
+        doAnswer(new Answer<DataTreeCandidateTip>() {
+            @Override
+            public DataTreeCandidateTip answer(final InvocationOnMock invocation) {
+                return actual.getCandidate();
+            }
+        }).when(cohort).getCandidate();
+
         return cohort;
     }
 
-    public static NormalizedNode<?,?> readStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id)
+    public static NormalizedNode<?,?> readStore(final TestActorRef<? extends Shard> shard, final YangInstanceIdentifier id)
             throws ExecutionException, InterruptedException {
-        return readStore(shard.underlyingActor().getDataStore(), id);
+        return readStore(shard.underlyingActor().getDataStore().getDataTree(), id);
     }
 
-    public static NormalizedNode<?,?> readStore(final InMemoryDOMDataStore store, final YangInstanceIdentifier id)
-            throws ExecutionException, InterruptedException {
-        DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
-
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future =
-            transaction.read(id);
-
-        Optional<NormalizedNode<?, ?>> optional = future.get();
-        NormalizedNode<?, ?> node = optional.isPresent()? optional.get() : null;
+    public static NormalizedNode<?,?> readStore(final DataTree store, final YangInstanceIdentifier id) {
+        final DataTreeSnapshot transaction = store.takeSnapshot();
 
-        transaction.close();
+        final Optional<NormalizedNode<?, ?>> optional = transaction.readNode(id);
+        final NormalizedNode<?, ?> node = optional.isPresent()? optional.get() : null;
 
         return node;
     }
 
     public static void writeToStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id,
-            final NormalizedNode<?,?> node) throws ExecutionException, InterruptedException {
+            final NormalizedNode<?,?> node) throws InterruptedException, ExecutionException {
         writeToStore(shard.underlyingActor().getDataStore(), id, node);
     }
 
-    public static void writeToStore(final InMemoryDOMDataStore store, final YangInstanceIdentifier id,
-            final NormalizedNode<?,?> node) throws ExecutionException, InterruptedException {
-        DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
+    public static void writeToStore(final ShardDataTree store, final YangInstanceIdentifier id,
+            final NormalizedNode<?,?> node) throws InterruptedException, ExecutionException {
+        final ReadWriteShardDataTreeTransaction transaction = store.newReadWriteTransaction("writeToStore", null);
 
-        transaction.write(id, node);
+        transaction.getSnapshot().write(id, node);
+        final ShardDataTreeCohort cohort = transaction.ready();
+        cohort.canCommit().get();
+        cohort.preCommit().get();
+        cohort.commit();
+    }
 
-        DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
-        commitCohort.preCommit().get();
-        commitCohort.commit().get();
+    public static void writeToStore(final DataTree store, final YangInstanceIdentifier id,
+            final NormalizedNode<?,?> node) throws DataValidationFailedException {
+        final DataTreeModification transaction = store.takeSnapshot().newModification();
+
+        transaction.write(id, node);
+        transaction.ready();
+        store.validate(transaction);
+        final DataTreeCandidate candidate = store.prepare(transaction);
+        store.commit(candidate);
     }
 
     @SuppressWarnings("serial")