Use YangInstanceIdentifier.EMPTY
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / AbstractShardTest.java
index 1100f3a7fa2c0fd584dc57618a6cd54339aa4b60..cc2e29e211ef590688fabcd876dc6bdc3be31974 100644 (file)
@@ -11,16 +11,23 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
+import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION;
 import akka.actor.ActorRef;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
 import akka.dispatch.Dispatchers;
 import akka.japi.Creator;
+import akka.pattern.Patterns;
 import akka.testkit.TestActorRef;
+import akka.util.Timeout;
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
+import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
 import java.util.Collections;
@@ -28,19 +35,29 @@ import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
+import org.opendaylight.controller.cluster.access.concepts.MemberName;
 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
+import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
+import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
+import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils;
+import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
+import org.opendaylight.controller.cluster.raft.Snapshot;
+import org.opendaylight.controller.cluster.raft.TestActorFactory;
 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
+import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
+import org.opendaylight.yangtools.concepts.Identifier;
 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;
@@ -48,11 +65,18 @@ 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.DataTreeCandidateNode;
 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.data.api.schema.tree.ModificationType;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
+import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
+import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Duration;
 
 /**
  * Abstract base for shard unit tests.
@@ -64,13 +88,15 @@ public abstract class AbstractShardTest extends AbstractActorTest{
 
     private static final AtomicInteger NEXT_SHARD_NUM = new AtomicInteger();
 
-    protected final ShardIdentifier shardID = ShardIdentifier.builder().memberName("member-1")
-            .shardName("inventory").type("config" + NEXT_SHARD_NUM.getAndIncrement()).build();
+    protected final ShardIdentifier shardID = ShardIdentifier.create("inventory", MemberName.forName("member-1"),
+        "config" + NEXT_SHARD_NUM.getAndIncrement());
 
     protected final Builder dataStoreContextBuilder = DatastoreContext.newBuilder().
             shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).
             shardHeartbeatIntervalInMillis(100);
 
+    protected final TestActorFactory actorFactory = new TestActorFactory(getSystem());
+
     @Before
     public void setUp() {
         InMemorySnapshotStore.clear();
@@ -81,6 +107,7 @@ public abstract class AbstractShardTest extends AbstractActorTest{
     public void tearDown() {
         InMemorySnapshotStore.clear();
         InMemoryJournal.clear();
+        actorFactory.close();
     }
 
     protected DatastoreContext newDatastoreContext() {
@@ -88,23 +115,25 @@ public abstract class AbstractShardTest extends AbstractActorTest{
     }
 
     protected Props newShardProps() {
-        return Shard.props(shardID, Collections.<String,String>emptyMap(),
-                newDatastoreContext(), SCHEMA_CONTEXT);
+        return newShardBuilder().props();
+    }
+
+    protected Shard.Builder newShardBuilder() {
+        return Shard.builder().id(shardID).datastoreContext(newDatastoreContext()).schemaContext(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")
-        Creator<Shard> creator = new Creator<Shard>() {
+        final Creator<Shard> creator = new Creator<Shard>() {
             @Override
             public Shard create() throws Exception {
-                return new Shard(shardID, Collections.<String,String>emptyMap(),
-                        newDatastoreContext(), SCHEMA_CONTEXT) {
+                return new Shard(newShardBuilder()) {
                     @Override
                     protected void onRecoveryComplete() {
                         try {
@@ -117,25 +146,25 @@ public abstract class AbstractShardTest extends AbstractActorTest{
             }
         };
 
-        TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+        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));
         }
@@ -155,7 +184,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();
@@ -179,9 +208,9 @@ public abstract class AbstractShardTest extends AbstractActorTest{
             final MutableCompositeModification modification,
             final Function<ShardDataTreeCohort, ListenableFuture<Void>> preCommit) {
 
-        ReadWriteShardDataTreeTransaction tx = dataStore.newReadWriteTransaction("setup-mock-" + cohortName, null);
+        final ReadWriteShardDataTreeTransaction tx = dataStore.newReadWriteTransaction("setup-mock-" + cohortName, null);
         tx.getSnapshot().write(path, data);
-        ShardDataTreeCohort cohort = createDelegatingMockCohort(cohortName, dataStore.finishTransaction(tx), preCommit);
+        final ShardDataTreeCohort cohort = createDelegatingMockCohort(cohortName, dataStore.finishTransaction(tx), preCommit);
 
         modification.addModification(new WriteModification(path, data));
 
@@ -196,7 +225,7 @@ public abstract class AbstractShardTest extends AbstractActorTest{
     protected ShardDataTreeCohort createDelegatingMockCohort(final String cohortName,
             final ShardDataTreeCohort actual,
             final Function<ShardDataTreeCohort, ListenableFuture<Void>> preCommit) {
-        ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, cohortName);
+        final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, cohortName);
 
         doAnswer(new Answer<ListenableFuture<Boolean>>() {
             @Override
@@ -240,31 +269,111 @@ public abstract class AbstractShardTest extends AbstractActorTest{
         return cohort;
     }
 
-    public static NormalizedNode<?,?> readStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id)
-            throws ExecutionException, InterruptedException {
-        return readStore(shard.underlyingActor().getDataStore().getDataTree(), id);
+    protected Object prepareReadyTransactionMessage(boolean remoteReadWriteTransaction, Shard shard, ShardDataTreeCohort cohort,
+                                                                  String transactionID,
+                                                                  MutableCompositeModification modification,
+                                                                  boolean doCommitOnReady) {
+        if(remoteReadWriteTransaction){
+            return prepareForwardedReadyTransaction(cohort, transactionID, CURRENT_VERSION,
+                    doCommitOnReady);
+        } else {
+            setupCohortDecorator(shard, cohort);
+            return prepareBatchedModifications(transactionID, modification, doCommitOnReady);
+        }
     }
 
-    public static NormalizedNode<?,?> readStore(final DataTree store, final YangInstanceIdentifier id) {
-        DataTreeSnapshot transaction = store.takeSnapshot();
+    protected ShardDataTreeCohort mockShardDataTreeCohort() {
+        ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class);
+        doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
+        doReturn(Futures.immediateFuture(null)).when(cohort).preCommit();
+        doReturn(Futures.immediateFuture(null)).when(cohort).commit();
+        doReturn(mockCandidate("candidate")).when(cohort).getCandidate();
+        return cohort;
+    }
+
+    static ShardDataTreeTransactionParent newShardDataTreeTransactionParent(ShardDataTreeCohort cohort) {
+        ShardDataTreeTransactionParent mockParent = mock(ShardDataTreeTransactionParent.class);
+        doReturn(cohort).when(mockParent).finishTransaction(any(ReadWriteShardDataTreeTransaction.class));
+        doNothing().when(mockParent).abortTransaction(any(AbstractShardDataTreeTransaction.class));
+        return mockParent;
+    }
+
+    protected ForwardedReadyTransaction prepareForwardedReadyTransaction(ShardDataTreeCohort cohort,
+            String transactionID, short version, boolean doCommitOnReady) {
+        return new ForwardedReadyTransaction(transactionID, version,
+                new ReadWriteShardDataTreeTransaction(newShardDataTreeTransactionParent(cohort), transactionID,
+                        mock(DataTreeModification.class)), doCommitOnReady);
+    }
+
+    protected Object prepareReadyTransactionMessage(boolean remoteReadWriteTransaction, Shard shard, ShardDataTreeCohort cohort,
+                                                                  String transactionID,
+                                                                  MutableCompositeModification modification) {
+        return prepareReadyTransactionMessage(remoteReadWriteTransaction, shard, cohort, transactionID, modification, false);
+    }
 
-        Optional<NormalizedNode<?, ?>> optional = transaction.readNode(id);
-        NormalizedNode<?, ?> node = optional.isPresent()? optional.get() : null;
+    protected void setupCohortDecorator(Shard shard, final ShardDataTreeCohort cohort) {
+        shard.getCommitCoordinator().setCohortDecorator(new ShardCommitCoordinator.CohortDecorator() {
+            @Override
+            public ShardDataTreeCohort decorate(Identifier transactionID, ShardDataTreeCohort actual) {
+                return cohort;
+            }
+        });
+    }
+
+    protected BatchedModifications prepareBatchedModifications(String transactionID,
+                                                               MutableCompositeModification modification) {
+        return prepareBatchedModifications(transactionID, modification, false);
+    }
+
+    private static BatchedModifications prepareBatchedModifications(String transactionID,
+                                                             MutableCompositeModification modification,
+                                                             boolean doCommitOnReady) {
+        final BatchedModifications batchedModifications = new BatchedModifications(transactionID, CURRENT_VERSION, null);
+        batchedModifications.addModification(modification);
+        batchedModifications.setReady(true);
+        batchedModifications.setDoCommitOnReady(doCommitOnReady);
+        batchedModifications.setTotalMessagesSent(1);
+        return batchedModifications;
+    }
 
-        return node;
+
+    public static NormalizedNode<?,?> readStore(final TestActorRef<? extends Shard> shard, final YangInstanceIdentifier id)
+            throws ExecutionException, InterruptedException {
+        return shard.underlyingActor().getDataStore().readNode(id).orNull();
+    }
+
+    public static NormalizedNode<?,?> readStore(final DataTree store, final YangInstanceIdentifier id) {
+        return store.takeSnapshot().readNode(id).orNull();
     }
 
     public static void writeToStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id,
             final NormalizedNode<?,?> node) throws InterruptedException, ExecutionException {
-        writeToStore(shard.underlyingActor().getDataStore(), id, node);
+        Future<Object> future = Patterns.ask(shard, newBatchedModifications("tx", id, node, true, true, 1),
+                new Timeout(5, TimeUnit.SECONDS));
+        try {
+            Await.ready(future, Duration.create(5, TimeUnit.SECONDS));
+        } catch(TimeoutException e) {
+            throw new ExecutionException(e);
+        }
     }
 
     public static void writeToStore(final ShardDataTree store, final YangInstanceIdentifier id,
             final NormalizedNode<?,?> node) throws InterruptedException, ExecutionException {
-        ReadWriteShardDataTreeTransaction transaction = store.newReadWriteTransaction("writeToStore", null);
+        final ReadWriteShardDataTreeTransaction transaction = store.newReadWriteTransaction("writeToStore", null);
 
         transaction.getSnapshot().write(id, node);
-        ShardDataTreeCohort cohort = transaction.ready();
+        final ShardDataTreeCohort cohort = transaction.ready();
+        cohort.canCommit().get();
+        cohort.preCommit().get();
+        cohort.commit();
+    }
+
+    public static void mergeToStore(final ShardDataTree store, final YangInstanceIdentifier id,
+            final NormalizedNode<?,?> node) throws InterruptedException, ExecutionException {
+        final ReadWriteShardDataTreeTransaction transaction = store.newReadWriteTransaction("writeToStore", null);
+
+        transaction.getSnapshot().merge(id, node);
+        final ShardDataTreeCohort cohort = transaction.ready();
         cohort.canCommit().get();
         cohort.preCommit().get();
         cohort.commit();
@@ -272,7 +381,7 @@ public abstract class AbstractShardTest extends AbstractActorTest{
 
     public static void writeToStore(final DataTree store, final YangInstanceIdentifier id,
             final NormalizedNode<?,?> node) throws DataValidationFailedException {
-        DataTreeModification transaction = store.takeSnapshot().newModification();
+        final DataTreeModification transaction = store.takeSnapshot().newModification();
 
         transaction.write(id, node);
         transaction.ready();
@@ -281,6 +390,84 @@ public abstract class AbstractShardTest extends AbstractActorTest{
         store.commit(candidate);
     }
 
+    DataTree setupInMemorySnapshotStore() throws DataValidationFailedException {
+        final DataTree testStore = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
+        testStore.setSchemaContext(SCHEMA_CONTEXT);
+
+        writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+
+        final NormalizedNode<?, ?> root = readStore(testStore, YangInstanceIdentifier.EMPTY);
+
+        InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create(
+                SerializationUtils.serializeNormalizedNode(root),
+                Collections.<ReplicatedLogEntry>emptyList(), 0, 1, -1, -1));
+        return testStore;
+    }
+
+    static DataTreeCandidatePayload payloadForModification(final DataTree source, final DataTreeModification mod) throws DataValidationFailedException {
+        source.validate(mod);
+        final DataTreeCandidate candidate = source.prepare(mod);
+        source.commit(candidate);
+        return DataTreeCandidatePayload.create(candidate);
+    }
+
+    static BatchedModifications newBatchedModifications(final String transactionID, final YangInstanceIdentifier path,
+            final NormalizedNode<?, ?> data, final boolean ready, final boolean doCommitOnReady, final int messagesSent) {
+        return newBatchedModifications(transactionID, null, path, data, ready, doCommitOnReady, messagesSent);
+    }
+
+    static BatchedModifications newBatchedModifications(final String transactionID, final String transactionChainID,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> data, final boolean ready, final boolean doCommitOnReady,
+            final int messagesSent) {
+        final BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, transactionChainID);
+        batched.addModification(new WriteModification(path, data));
+        batched.setReady(ready);
+        batched.setDoCommitOnReady(doCommitOnReady);
+        batched.setTotalMessagesSent(messagesSent);
+        return batched;
+    }
+
+    @SuppressWarnings("unchecked")
+    static void verifyOuterListEntry(final TestActorRef<Shard> shard, final Object expIDValue) throws Exception {
+        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);
+        final Object entry = ((Iterable<Object>)outerList.getValue()).iterator().next();
+        assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
+                entry instanceof MapEntryNode);
+        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());
+        assertEquals(TestModel.ID_QNAME.getLocalName() + " value", expIDValue, idLeaf.get().getValue());
+    }
+
+    public static DataTreeCandidateTip mockCandidate(final String name) {
+        final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name);
+        final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node");
+        doReturn(ModificationType.WRITE).when(mockCandidateNode).getModificationType();
+        doReturn(Optional.of(ImmutableNodes.containerNode(CarsModel.CARS_QNAME))).when(mockCandidateNode).getDataAfter();
+        doReturn(CarsModel.BASE_PATH).when(mockCandidate).getRootPath();
+        doReturn(mockCandidateNode).when(mockCandidate).getRootNode();
+        return mockCandidate;
+    }
+
+    static DataTreeCandidateTip mockUnmodifiedCandidate(final String name) {
+        final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name);
+        final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node");
+        doReturn(ModificationType.UNMODIFIED).when(mockCandidateNode).getModificationType();
+        doReturn(YangInstanceIdentifier.EMPTY).when(mockCandidate).getRootPath();
+        doReturn(mockCandidateNode).when(mockCandidate).getRootNode();
+        return mockCandidate;
+    }
+
+    static void commitTransaction(final DataTree store, final DataTreeModification modification) throws DataValidationFailedException {
+        modification.ready();
+        store.validate(modification);
+        store.commit(store.prepare(modification));
+    }
+
     @SuppressWarnings("serial")
     public static final class DelegatingShardCreator implements Creator<Shard> {
         private final Creator<Shard> delegate;