X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Ftest%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FShardTest.java;h=83b15b99df43f74e0a8a5d61d7493b1792c17e8b;hp=cc96d0d3b0d070623c737dc8f78340c45a20539f;hb=340a2d4c979ac6f8d5adff8bd9e1c9f724e7a164;hpb=3940ce6060e027fe870244346e5309229cc8dc48 diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java index cc96d0d3b0..83b15b99df 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java @@ -4,44 +4,45 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION; import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.PoisonPill; import akka.actor.Props; +import akka.actor.Status.Failure; import akka.dispatch.Dispatchers; import akka.dispatch.OnComplete; import akka.japi.Creator; -import akka.japi.Procedure; import akka.pattern.Patterns; -import akka.persistence.SnapshotSelectionCriteria; +import akka.persistence.SaveSnapshotSuccess; 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.MoreExecutors; import com.google.common.util.concurrent.Uninterruptibles; import java.io.IOException; import java.util.Collections; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import org.junit.Test; import org.mockito.InOrder; import org.opendaylight.controller.cluster.DataPersistenceProvider; +import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; +import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats; import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction; import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications; @@ -55,9 +56,13 @@ import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTran import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved; import org.opendaylight.controller.cluster.datastore.messages.ReadData; import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply; +import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction; import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener; import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply; +import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListener; +import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListenerReply; +import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged; import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext; import org.opendaylight.controller.cluster.datastore.modification.DeleteModification; import org.opendaylight.controller.cluster.datastore.modification.MergeModification; @@ -65,8 +70,8 @@ import org.opendaylight.controller.cluster.datastore.modification.Modification; import org.opendaylight.controller.cluster.datastore.modification.ModificationPayload; import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification; import org.opendaylight.controller.cluster.datastore.modification.WriteModification; -import org.opendaylight.controller.cluster.datastore.utils.MessageCollectorActor; import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener; +import org.opendaylight.controller.cluster.datastore.utils.MockDataTreeChangeListener; import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils; import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener; import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListenerReply; @@ -75,62 +80,97 @@ import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry; import org.opendaylight.controller.cluster.raft.Snapshot; import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries; -import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus; import org.opendaylight.controller.cluster.raft.client.messages.FindLeader; import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply; +import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal; import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore; +import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor; import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper; import org.opendaylight.controller.md.cluster.datastore.model.TestModel; import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker; import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException; -import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore; -import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory; import org.opendaylight.controller.protobuff.messages.cohort3pc.ThreePhaseCommitCohortMessages; import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages.CreateTransactionReply; -import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort; -import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction; +import org.opendaylight.yangtools.yang.common.QName; 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.ContainerNode; 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.MapNode; 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.DataTreeCandidates; +import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification; +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.impl.schema.ImmutableNodes; +import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder; +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.FiniteDuration; public class ShardTest extends AbstractShardTest { + private static final QName CARS_QNAME = QName.create("urn:opendaylight:params:xml:ns:yang:controller:md:sal:dom:store:test:cars", "2014-03-13", "cars"); + + private static final String DUMMY_DATA = "Dummy data as snapshot sequence number is set to 0 in InMemorySnapshotStore and journal recovery seq number will start from 1"; + + final CountDownLatch recoveryComplete = new CountDownLatch(1); + + protected Props newShardPropsWithRecoveryComplete() { + + final Creator creator = new Creator() { + @Override + public Shard create() throws Exception { + return new Shard(shardID, Collections.emptyMap(), + newDatastoreContext(), SCHEMA_CONTEXT) { + @Override + protected void onRecoveryComplete() { + try { + super.onRecoveryComplete(); + } finally { + recoveryComplete.countDown(); + } + } + }; + } + }; + return Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()); + } @Test public void testRegisterChangeListener() throws Exception { new ShardTestKit(getSystem()) {{ - TestActorRef shard = TestActorRef.create(getSystem(), + final TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), "testRegisterChangeListener"); waitUntilLeader(shard); shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender()); - MockDataChangeListener listener = new MockDataChangeListener(1); - ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener), + final MockDataChangeListener listener = new MockDataChangeListener(1); + final ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener), "testRegisterChangeListener-DataChangeListener"); shard.tell(new RegisterChangeListener(TestModel.TEST_PATH, dclActor, AsyncDataBroker.DataChangeScope.BASE), getRef()); - RegisterChangeListenerReply reply = expectMsgClass(duration("3 seconds"), + final RegisterChangeListenerReply reply = expectMsgClass(duration("3 seconds"), RegisterChangeListenerReply.class); - String replyPath = reply.getListenerRegistrationPath().toString(); + final String replyPath = reply.getListenerRegistrationPath().toString(); assertTrue("Incorrect reply path: " + replyPath, replyPath.matches( "akka:\\/\\/test\\/user\\/testRegisterChangeListener\\/\\$.*")); - YangInstanceIdentifier path = TestModel.TEST_PATH; + final YangInstanceIdentifier path = TestModel.TEST_PATH; writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); listener.waitForChangeEvents(path); @@ -155,7 +195,7 @@ public class ShardTest extends AbstractShardTest { // ElectionTimeout with the sending of the RegisterChangeListener message. final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1); final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1); - Creator creator = new Creator() { + final Creator creator = new Creator() { boolean firstElectionTimeout = true; @Override @@ -197,16 +237,16 @@ public class ShardTest extends AbstractShardTest { } }; - MockDataChangeListener listener = new MockDataChangeListener(1); - ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener), + final MockDataChangeListener listener = new MockDataChangeListener(1); + final ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener), "testRegisterChangeListenerWhenNotLeaderInitially-DataChangeListener"); - TestActorRef shard = TestActorRef.create(getSystem(), + final TestActorRef shard = TestActorRef.create(getSystem(), Props.create(new DelegatingShardCreator(creator)), "testRegisterChangeListenerWhenNotLeaderInitially"); // Write initial data into the in-memory store. - YangInstanceIdentifier path = TestModel.TEST_PATH; + final YangInstanceIdentifier path = TestModel.TEST_PATH; writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); // Wait until the shard receives the first ElectionTimeout message. @@ -217,13 +257,13 @@ public class ShardTest extends AbstractShardTest { shard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.SUBTREE), getRef()); - RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"), + final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"), RegisterChangeListenerReply.class); assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath()); // Sanity check - verify the shard is not the leader yet. shard.tell(new FindLeader(), getRef()); - FindLeaderReply findLeadeReply = + final FindLeaderReply findLeadeReply = expectMsgClass(duration("5 seconds"), FindLeaderReply.class); assertNull("Expected the shard not to be the leader", findLeadeReply.getLeaderActor()); @@ -240,22 +280,126 @@ public class ShardTest extends AbstractShardTest { }}; } + @Test + public void testRegisterDataTreeChangeListener() throws Exception { + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps(), "testRegisterDataTreeChangeListener"); + + waitUntilLeader(shard); + + shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender()); + + final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1); + final ActorRef dclActor = getSystem().actorOf(DataTreeChangeListenerActor.props(listener), + "testRegisterDataTreeChangeListener-DataTreeChangeListener"); + + shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor), getRef()); + + final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("3 seconds"), + RegisterDataTreeChangeListenerReply.class); + final String replyPath = reply.getListenerRegistrationPath().toString(); + assertTrue("Incorrect reply path: " + replyPath, replyPath.matches( + "akka:\\/\\/test\\/user\\/testRegisterDataTreeChangeListener\\/\\$.*")); + + final YangInstanceIdentifier path = TestModel.TEST_PATH; + writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + + listener.waitForChangeEvents(); + + dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender()); + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @SuppressWarnings("serial") + @Test + public void testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception { + new ShardTestKit(getSystem()) {{ + final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1); + final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1); + final Creator creator = new Creator() { + boolean firstElectionTimeout = true; + + @Override + public Shard create() throws Exception { + return new Shard(shardID, Collections.emptyMap(), + dataStoreContextBuilder.persistent(false).build(), SCHEMA_CONTEXT) { + @Override + public void onReceiveCommand(final Object message) throws Exception { + if(message instanceof ElectionTimeout && firstElectionTimeout) { + firstElectionTimeout = false; + final ActorRef self = getSelf(); + new Thread() { + @Override + public void run() { + Uninterruptibles.awaitUninterruptibly( + onChangeListenerRegistered, 5, TimeUnit.SECONDS); + self.tell(message, self); + } + }.start(); + + onFirstElectionTimeout.countDown(); + } else { + super.onReceiveCommand(message); + } + } + }; + } + }; + + final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1); + final ActorRef dclActor = getSystem().actorOf(DataTreeChangeListenerActor.props(listener), + "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener"); + + final TestActorRef shard = TestActorRef.create(getSystem(), + Props.create(new DelegatingShardCreator(creator)), + "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration"); + + final YangInstanceIdentifier path = TestModel.TEST_PATH; + writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + + assertEquals("Got first ElectionTimeout", true, + onFirstElectionTimeout.await(5, TimeUnit.SECONDS)); + + shard.tell(new RegisterDataTreeChangeListener(path, dclActor), getRef()); + final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"), + RegisterDataTreeChangeListenerReply.class); + assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath()); + + shard.tell(new FindLeader(), getRef()); + final FindLeaderReply findLeadeReply = + expectMsgClass(duration("5 seconds"), FindLeaderReply.class); + assertNull("Expected the shard not to be the leader", findLeadeReply.getLeaderActor()); + + writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + + onChangeListenerRegistered.countDown(); + + // TODO: investigate why we do not receive data chage events + listener.waitForChangeEvents(); + + dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender()); + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + @Test public void testCreateTransaction(){ new ShardTestKit(getSystem()) {{ - ActorRef shard = getSystem().actorOf(newShardProps(), "testCreateTransaction"); + final ActorRef shard = getSystem().actorOf(newShardProps(), "testCreateTransaction"); waitUntilLeader(shard); shard.tell(new UpdateSchemaContext(TestModel.createTestContext()), getRef()); shard.tell(new CreateTransaction("txn-1", - TransactionProxy.TransactionType.READ_ONLY.ordinal() ).toSerializable(), getRef()); + TransactionType.READ_ONLY.ordinal() ).toSerializable(), getRef()); - CreateTransactionReply reply = expectMsgClass(duration("3 seconds"), + final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class); - String path = reply.getTransactionActorPath().toString(); + final String path = reply.getTransactionActorPath().toString(); assertTrue("Unexpected transaction path " + path, path.contains("akka://test/user/testCreateTransaction/shard-txn-1")); @@ -271,13 +415,13 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); shard.tell(new CreateTransaction("txn-1", - TransactionProxy.TransactionType.READ_ONLY.ordinal() , "foobar").toSerializable(), + TransactionType.READ_ONLY.ordinal() , "foobar").toSerializable(), getRef()); - CreateTransactionReply reply = expectMsgClass(duration("3 seconds"), + final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class); - String path = reply.getTransactionActorPath().toString(); + final String path = reply.getTransactionActorPath().toString(); assertTrue("Unexpected transaction path " + path, path.contains("akka://test/user/testCreateTransactionOnChain/shard-txn-1")); @@ -322,7 +466,7 @@ public class ShardTest extends AbstractShardTest { assertEquals("Recovery complete", true, Uninterruptibles.awaitUninterruptibly(recoveryComplete, 5, TimeUnit.SECONDS)); - String address = "akka://foobar"; + final String address = "akka://foobar"; shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID.toString(), address)); assertEquals("getPeerAddresses", address, @@ -334,24 +478,28 @@ public class ShardTest extends AbstractShardTest { @Test public void testApplySnapshot() throws Exception { - TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), + final TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), "testApplySnapshot"); - InMemoryDOMDataStore store = new InMemoryDOMDataStore("OPER", MoreExecutors.sameThreadExecutor()); - store.onGlobalContextUpdated(SCHEMA_CONTEXT); + final DataTree store = InMemoryDataTreeFactory.getInstance().create(); + store.setSchemaContext(SCHEMA_CONTEXT); + + final ContainerNode container = ImmutableContainerNodeBuilder.create().withNodeIdentifier( + new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME)). + withChild(ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).addChild( + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1)).build()).build(); - writeToStore(store, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + writeToStore(store, TestModel.TEST_PATH, container); - YangInstanceIdentifier root = YangInstanceIdentifier.builder().build(); - NormalizedNode expected = readStore(store, root); + final YangInstanceIdentifier root = YangInstanceIdentifier.builder().build(); + final NormalizedNode expected = readStore(store, root); - ApplySnapshot applySnapshot = new ApplySnapshot(Snapshot.create( - SerializationUtils.serializeNormalizedNode(expected), - Collections.emptyList(), 1, 2, 3, 4)); + final Snapshot snapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(expected), + Collections.emptyList(), 1, 2, 3, 4); - shard.underlyingActor().onReceiveCommand(applySnapshot); + shard.underlyingActor().getRaftActorSnapshotCohort().applySnapshot(snapshot.getState()); - NormalizedNode actual = readStore(shard, root); + final NormalizedNode actual = readStore(shard, root); assertEquals("Root node", expected, actual); @@ -361,73 +509,142 @@ public class ShardTest extends AbstractShardTest { @Test public void testApplyState() throws Exception { - TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), "testApplyState"); + final TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), "testApplyState"); - NormalizedNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + final NormalizedNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME); - ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2, + final ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2, newModificationPayload(new WriteModification(TestModel.TEST_PATH, node)))); shard.underlyingActor().onReceiveCommand(applyState); - NormalizedNode actual = readStore(shard, TestModel.TEST_PATH); + final NormalizedNode actual = readStore(shard, TestModel.TEST_PATH); assertEquals("Applied state", node, actual); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); } @Test - public void testRecovery() throws Exception { + public void testApplyStateWithCandidatePayload() throws Exception { - // Set up the InMemorySnapshotStore. + final TestActorRef shard = TestActorRef.create(getSystem(), newShardPropsWithRecoveryComplete(), "testApplyState"); + + recoveryComplete.await(5, TimeUnit.SECONDS); + + final NormalizedNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + final DataTreeCandidate candidate = DataTreeCandidates.fromNormalizedNode(TestModel.TEST_PATH, node); + + final ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2, + DataTreeCandidatePayload.create(candidate))); - InMemoryDOMDataStore testStore = InMemoryDOMDataStoreFactory.create("Test", null, null); - testStore.onGlobalContextUpdated(SCHEMA_CONTEXT); + shard.underlyingActor().onReceiveCommand(applyState); + + final NormalizedNode actual = readStore(shard, TestModel.TEST_PATH); + assertEquals("Applied state", node, actual); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + } + + DataTree setupInMemorySnapshotStore() throws DataValidationFailedException { + final DataTree testStore = InMemoryDataTreeFactory.getInstance().create(); + testStore.setSchemaContext(SCHEMA_CONTEXT); writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - NormalizedNode root = readStore(testStore, YangInstanceIdentifier.builder().build()); + final NormalizedNode root = readStore(testStore, YangInstanceIdentifier.builder().build()); InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create( SerializationUtils.serializeNormalizedNode(root), Collections.emptyList(), 0, 1, -1, -1)); + return testStore; + } + + private 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); + } + + @Test + public void testDataTreeCandidateRecovery() throws Exception { + // Set up the InMemorySnapshotStore. + final DataTree source = setupInMemorySnapshotStore(); + + final DataTreeModification writeMod = source.takeSnapshot().newModification(); + writeMod.write(TestModel.OUTER_LIST_PATH, ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build()); + writeMod.ready(); + InMemoryJournal.addEntry(shardID.toString(), 0, DUMMY_DATA); + + // Set up the InMemoryJournal. + InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, payloadForModification(source, writeMod))); + + final int nListEntries = 16; + final Set listEntryKeys = new HashSet<>(); + + // Add some ModificationPayload entries + for (int i = 1; i <= nListEntries; i++) { + listEntryKeys.add(Integer.valueOf(i)); + + final YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) + .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build(); + + final DataTreeModification mod = source.takeSnapshot().newModification(); + mod.merge(path, ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i)); + mod.ready(); + InMemoryJournal.addEntry(shardID.toString(), i+1, new ReplicatedLogImplEntry(i, 1, + payloadForModification(source, mod))); + } + + InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2, + new ApplyJournalEntries(nListEntries)); + + testRecovery(listEntryKeys); + } + + @Test + public void testModicationRecovery() throws Exception { + + // Set up the InMemorySnapshotStore. + setupInMemorySnapshotStore(); // Set up the InMemoryJournal. - InMemoryJournal.addEntry(shardID.toString(), 0, new ReplicatedLogImplEntry(0, 1, newModificationPayload( + InMemoryJournal.addEntry(shardID.toString(), 0, DUMMY_DATA); + + InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, newModificationPayload( new WriteModification(TestModel.OUTER_LIST_PATH, ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build())))); - int nListEntries = 16; - Set listEntryKeys = new HashSet<>(); + final int nListEntries = 16; + final Set listEntryKeys = new HashSet<>(); // Add some ModificationPayload entries for(int i = 1; i <= nListEntries; i++) { listEntryKeys.add(Integer.valueOf(i)); - YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) + final YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build(); - Modification mod = new MergeModification(path, + final Modification mod = new MergeModification(path, ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i)); - InMemoryJournal.addEntry(shardID.toString(), i, new ReplicatedLogImplEntry(i, 1, + InMemoryJournal.addEntry(shardID.toString(), i + 1, new ReplicatedLogImplEntry(i, 1, newModificationPayload(mod))); } - InMemoryJournal.addEntry(shardID.toString(), nListEntries + 1, + InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2, new ApplyJournalEntries(nListEntries)); testRecovery(listEntryKeys); } - private ModificationPayload newModificationPayload(final Modification... mods) throws IOException { - MutableCompositeModification compMod = new MutableCompositeModification(); - for(Modification mod: mods) { + private static ModificationPayload newModificationPayload(final Modification... mods) throws IOException { + final MutableCompositeModification compMod = new MutableCompositeModification(); + for(final Modification mod: mods) { compMod.addModification(mod); } return new ModificationPayload(compMod); } - @SuppressWarnings({ "unchecked" }) @Test public void testConcurrentThreePhaseCommits() throws Throwable { new ShardTestKit(getSystem()) {{ @@ -437,68 +654,67 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionID1 = "tx1"; - final String transactionID2 = "tx2"; - final String transactionID3 = "tx3"; + // Setup 3 simulated transactions with mock cohorts backed by real cohorts. - final AtomicReference mockCohort1 = new AtomicReference<>(); - final AtomicReference mockCohort2 = new AtomicReference<>(); - final AtomicReference mockCohort3 = new AtomicReference<>(); - ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() { - @Override - public DOMStoreThreePhaseCommitCohort decorate(String transactionID, DOMStoreThreePhaseCommitCohort actual) { - if(transactionID.equals(transactionID1)) { - mockCohort1.set(createDelegatingMockCohort("cohort1", actual)); - return mockCohort1.get(); - } else if(transactionID.equals(transactionID2)) { - mockCohort2.set(createDelegatingMockCohort("cohort2", actual)); - return mockCohort2.get(); - } else { - mockCohort3.set(createDelegatingMockCohort("cohort3", actual)); - return mockCohort3.get(); - } - } - }; + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator); + final String transactionID1 = "tx1"; + final MutableCompositeModification modification1 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, + TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1); - long timeoutSec = 5; + final String transactionID2 = "tx2"; + final MutableCompositeModification modification2 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore, + TestModel.OUTER_LIST_PATH, + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), + modification2); + + final String transactionID3 = "tx3"; + final MutableCompositeModification modification3 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore, + YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) + .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(), + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), + modification3); + + final long timeoutSec = 5; final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS); final Timeout timeout = new Timeout(duration); - // Send a BatchedModifications message for the first transaction. + // Simulate the ForwardedReadyTransaction message for the first Tx that would be sent + // by the ShardTransaction. - shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - BatchedModificationsReply batchedReply = expectMsgClass(duration, BatchedModificationsReply.class); - assertEquals("getCohortPath", shard.path().toString(), batchedReply.getCohortPath()); - assertEquals("getNumBatched", 1, batchedReply.getNumBatched()); + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort1, modification1, true, false), getRef()); + final ReadyTransactionReply readyReply = ReadyTransactionReply.fromSerializable( + expectMsgClass(duration, ReadyTransactionReply.class)); + assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath()); // Send the CanCommitTransaction message for the first Tx. shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); - // Send BatchedModifications for the next 2 Tx's. + // Send the ForwardedReadyTransaction for the next 2 Tx's. - shard.tell(newBatchedModifications(transactionID2, TestModel.OUTER_LIST_PATH, - ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort2, modification2, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(newBatchedModifications(transactionID3, YangInstanceIdentifier.builder( - TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(), - ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + shard.tell(new ForwardedReadyTransaction(transactionID3, CURRENT_VERSION, + cohort3, modification3, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and // processed after the first Tx completes. - Future canCommitFuture1 = Patterns.ask(shard, + final Future canCommitFuture1 = Patterns.ask(shard, new CanCommitTransaction(transactionID2).toSerializable(), timeout); - Future canCommitFuture2 = Patterns.ask(shard, + final Future canCommitFuture2 = Patterns.ask(shard, new CanCommitTransaction(transactionID3).toSerializable(), timeout); // Send the CommitTransaction message for the first Tx. After it completes, it should @@ -527,7 +743,7 @@ public class ShardTest extends AbstractShardTest { try { assertEquals("Commit response type", expRespType, resp.getClass()); onSuccess(resp); - } catch (Exception e) { + } catch (final Exception e) { caughtEx.set(e); } } @@ -559,11 +775,11 @@ public class ShardTest extends AbstractShardTest { @Override void onSuccess(final Object resp) throws Exception { - CanCommitTransactionReply canCommitReply = + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(resp); assertEquals("Can commit", true, canCommitReply.getCanCommit()); - Future commitFuture = Patterns.ask(shard, + final Future commitFuture = Patterns.ask(shard, new CommitTransaction(transactionID).toSerializable(), timeout); commitFuture.onComplete(new OnCommitFutureComplete(), getSystem().dispatcher()); } @@ -575,7 +791,7 @@ public class ShardTest extends AbstractShardTest { canCommitFuture2.onComplete(new OnCanCommitFutureComplete(transactionID3), getSystem().dispatcher()); - boolean done = commitLatch.await(timeoutSec, TimeUnit.SECONDS); + final boolean done = commitLatch.await(timeoutSec, TimeUnit.SECONDS); if(caughtEx.get() != null) { throw caughtEx.get(); @@ -583,31 +799,20 @@ public class ShardTest extends AbstractShardTest { assertEquals("Commits complete", true, done); - InOrder inOrder = inOrder(mockCohort1.get(), mockCohort2.get(), mockCohort3.get()); - inOrder.verify(mockCohort1.get()).canCommit(); - inOrder.verify(mockCohort1.get()).preCommit(); - inOrder.verify(mockCohort1.get()).commit(); - inOrder.verify(mockCohort2.get()).canCommit(); - inOrder.verify(mockCohort2.get()).preCommit(); - inOrder.verify(mockCohort2.get()).commit(); - inOrder.verify(mockCohort3.get()).canCommit(); - inOrder.verify(mockCohort3.get()).preCommit(); - inOrder.verify(mockCohort3.get()).commit(); + final InOrder inOrder = inOrder(cohort1, cohort2, cohort3); + inOrder.verify(cohort1).canCommit(); + inOrder.verify(cohort1).preCommit(); + inOrder.verify(cohort1).commit(); + inOrder.verify(cohort2).canCommit(); + inOrder.verify(cohort2).preCommit(); + inOrder.verify(cohort2).commit(); + inOrder.verify(cohort3).canCommit(); + inOrder.verify(cohort3).preCommit(); + inOrder.verify(cohort3).commit(); // Verify data in the data store. - 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); - Object entry = ((Iterable)outerList.getValue()).iterator().next(); - assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode", - entry instanceof MapEntryNode); - MapEntryNode mapEntry = (MapEntryNode)entry; - Optional> idLeaf = - mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME)); - assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent()); - assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue()); + verifyOuterListEntry(shard, 1); verifyLastApplied(shard, 2); @@ -615,36 +820,38 @@ public class ShardTest extends AbstractShardTest { }}; } - private BatchedModifications newBatchedModifications(String transactionID, YangInstanceIdentifier path, - NormalizedNode data, boolean ready) { - return newBatchedModifications(transactionID, null, path, data, ready); + private 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); } - private BatchedModifications newBatchedModifications(String transactionID, String transactionChainID, - YangInstanceIdentifier path, NormalizedNode data, boolean ready) { - BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, transactionChainID); + private 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") @Test - public void testMultipleBatchedModifications() throws Throwable { + public void testBatchedModificationsWithNoCommitOnReady() throws Throwable { new ShardTestKit(getSystem()) {{ final TestActorRef shard = TestActorRef.create(getSystem(), newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testMultipleBatchedModifications"); + "testBatchedModificationsWithNoCommitOnReady"); waitUntilLeader(shard); final String transactionID = "tx"; - FiniteDuration duration = duration("5 seconds"); + final FiniteDuration duration = duration("5 seconds"); - final AtomicReference mockCohort = new AtomicReference<>(); - ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() { + final AtomicReference mockCohort = new AtomicReference<>(); + final ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() { @Override - public DOMStoreThreePhaseCommitCohort decorate(String txID, DOMStoreThreePhaseCommitCohort actual) { + public ShardDataTreeCohort decorate(final String txID, final ShardDataTreeCohort actual) { if(mockCohort.get() == null) { mockCohort.set(createDelegatingMockCohort("cohort", actual)); } @@ -658,24 +865,24 @@ public class ShardTest extends AbstractShardTest { // Send a BatchedModifications to start a transaction. shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef()); + ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), getRef()); expectMsgClass(duration, BatchedModificationsReply.class); // Send a couple more BatchedModifications. shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH, - ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false), getRef()); + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef()); expectMsgClass(duration, BatchedModificationsReply.class); shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder( TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(), - ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true, false, 3), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message. shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); @@ -684,30 +891,156 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef()); expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); - InOrder inOrder = inOrder(mockCohort.get()); + final InOrder inOrder = inOrder(mockCohort.get()); + inOrder.verify(mockCohort.get()).canCommit(); + inOrder.verify(mockCohort.get()).preCommit(); + inOrder.verify(mockCohort.get()).commit(); + + // Verify data in the data store. + + verifyOuterListEntry(shard, 1); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testBatchedModificationsWithCommitOnReady() throws Throwable { + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testBatchedModificationsWithCommitOnReady"); + + waitUntilLeader(shard); + + final String transactionID = "tx"; + final FiniteDuration duration = duration("5 seconds"); + + final AtomicReference mockCohort = new AtomicReference<>(); + final ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() { + @Override + public ShardDataTreeCohort decorate(final String txID, final ShardDataTreeCohort actual) { + if(mockCohort.get() == null) { + mockCohort.set(createDelegatingMockCohort("cohort", actual)); + } + + return mockCohort.get(); + } + }; + + shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator); + + // Send a BatchedModifications to start a transaction. + + shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), getRef()); + expectMsgClass(duration, BatchedModificationsReply.class); + + // Send a couple more BatchedModifications. + + shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH, + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef()); + expectMsgClass(duration, BatchedModificationsReply.class); + + shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder( + TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(), + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true, true, 3), getRef()); + + expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); + + final InOrder inOrder = inOrder(mockCohort.get()); inOrder.verify(mockCohort.get()).canCommit(); inOrder.verify(mockCohort.get()).preCommit(); inOrder.verify(mockCohort.get()).commit(); // Verify data in the data store. - 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); - Object entry = ((Iterable)outerList.getValue()).iterator().next(); - assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode", - entry instanceof MapEntryNode); - MapEntryNode mapEntry = (MapEntryNode)entry; - Optional> idLeaf = - mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME)); - assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent()); - assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue()); + verifyOuterListEntry(shard, 1); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test(expected=IllegalStateException.class) + public void testBatchedModificationsReadyWithIncorrectTotalMessageCount() throws Throwable { + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testBatchedModificationsReadyWithIncorrectTotalMessageCount"); + + waitUntilLeader(shard); + + final String transactionID = "tx1"; + final BatchedModifications batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION, null); + batched.setReady(true); + batched.setTotalMessagesSent(2); + + shard.tell(batched, getRef()); + + final Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + + if(failure != null) { + throw failure.cause(); + } + }}; + } + + @Test + public void testBatchedModificationsWithOperationFailure() throws Throwable { + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testBatchedModificationsWithOperationFailure"); + + waitUntilLeader(shard); + + // Test merge with invalid data. An exception should occur when the merge is applied. Note that + // write will not validate the children for performance reasons. + + String transactionID = "tx1"; + + ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier( + new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME)). + withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build(); + + BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, null); + batched.addModification(new MergeModification(TestModel.TEST_PATH, invalidData)); + shard.tell(batched, getRef()); + Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class); + + Throwable cause = failure.cause(); + + batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION, null); + batched.setReady(true); + batched.setTotalMessagesSent(2); + + shard.tell(batched, getRef()); + + failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class); + assertEquals("Failure cause", cause, failure.cause()); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; } + @SuppressWarnings("unchecked") + private static void verifyOuterListEntry(final TestActorRef 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)outerList.getValue()).iterator().next(); + assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode", + entry instanceof MapEntryNode); + final MapEntryNode mapEntry = (MapEntryNode)entry; + final Optional> 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()); + } + @Test public void testBatchedModificationsOnTransactionChain() throws Throwable { new ShardTestKit(getSystem()) {{ @@ -717,35 +1050,35 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - String transactionChainID = "txChain"; - String transactionID1 = "tx1"; - String transactionID2 = "tx2"; + final String transactionChainID = "txChain"; + final String transactionID1 = "tx1"; + final String transactionID2 = "tx2"; - FiniteDuration duration = duration("5 seconds"); + final FiniteDuration duration = duration("5 seconds"); // Send a BatchedModifications to start a chained write transaction and ready it. - ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); - YangInstanceIdentifier path = TestModel.TEST_PATH; + final ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + final YangInstanceIdentifier path = TestModel.TEST_PATH; shard.tell(newBatchedModifications(transactionID1, transactionChainID, path, - containerNode, true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + containerNode, true, false, 1), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // Create a read Tx on the same chain. - shard.tell(new CreateTransaction(transactionID2, TransactionProxy.TransactionType.READ_ONLY.ordinal() , + shard.tell(new CreateTransaction(transactionID2, TransactionType.READ_ONLY.ordinal() , transactionChainID).toSerializable(), getRef()); - CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class); + final CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class); getSystem().actorSelection(createReply.getTransactionActorPath()).tell(new ReadData(path), getRef()); - ReadDataReply readReply = expectMsgClass(duration("3 seconds"), ReadDataReply.class); + final ReadDataReply readReply = expectMsgClass(duration("3 seconds"), ReadDataReply.class); assertEquals("Read node", containerNode, readReply.getNormalizedNode()); // Commit the write transaction. shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); @@ -754,7 +1087,7 @@ public class ShardTest extends AbstractShardTest { // Verify data in the data store. - NormalizedNode actualNode = readStore(shard, path); + final NormalizedNode actualNode = readStore(shard, path); assertEquals("Stored node", containerNode, actualNode); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); @@ -765,7 +1098,9 @@ public class ShardTest extends AbstractShardTest { public void testOnBatchedModificationsWhenNotLeader() { final AtomicBoolean overrideLeaderCalls = new AtomicBoolean(); new ShardTestKit(getSystem()) {{ - Creator creator = new Creator() { + final Creator creator = new Creator() { + private static final long serialVersionUID = 1L; + @Override public Shard create() throws Exception { return new Shard(shardID, Collections.emptyMap(), @@ -784,14 +1119,14 @@ public class ShardTest extends AbstractShardTest { } }; - TestActorRef shard = TestActorRef.create(getSystem(), + final TestActorRef shard = TestActorRef.create(getSystem(), Props.create(new DelegatingShardCreator(creator)), "testOnBatchedModificationsWhenNotLeader"); waitUntilLeader(shard); overrideLeaderCalls.set(true); - BatchedModifications batched = new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, ""); + final BatchedModifications batched = new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, ""); shard.tell(batched, ActorRef.noSender()); @@ -802,43 +1137,194 @@ public class ShardTest extends AbstractShardTest { } @Test - public void testCommitWithPersistenceDisabled() throws Throwable { - dataStoreContextBuilder.persistent(false); + public void testForwardedReadyTransactionWithImmediateCommit() throws Exception{ new ShardTestKit(getSystem()) {{ final TestActorRef shard = TestActorRef.create(getSystem(), newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testCommitWithPersistenceDisabled"); + "testForwardedReadyTransactionWithImmediateCommit"); waitUntilLeader(shard); - String transactionID = "tx"; - FiniteDuration duration = duration("5 seconds"); + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - // Send a BatchedModifications to start a transaction. + final String transactionID = "tx1"; + final MutableCompositeModification modification = new MutableCompositeModification(); + final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore, + TestModel.TEST_PATH, containerNode, modification); - NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); - shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH, containerNode, true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + final FiniteDuration duration = duration("5 seconds"); - // Send the CanCommitTransaction message. + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. - shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( - expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); - assertEquals("Can commit", true, canCommitReply.getCanCommit()); + shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, + cohort, modification, true, true), getRef()); + + expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class); + + final InOrder inOrder = inOrder(cohort); + inOrder.verify(cohort).canCommit(); + inOrder.verify(cohort).preCommit(); + inOrder.verify(cohort).commit(); + + final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); + assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testReadyLocalTransactionWithImmediateCommit() throws Exception{ + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testReadyLocalTransactionWithImmediateCommit"); + + waitUntilLeader(shard); + + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); + + final DataTreeModification modification = dataStore.getDataTree().takeSnapshot().newModification(); + + final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + new WriteModification(TestModel.TEST_PATH, writeData).apply(modification); + final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(); + new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification); + + final String txId = "tx1"; + modification.ready(); + final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, true); + + shard.tell(readyMessage, getRef()); + + expectMsgClass(CommitTransactionReply.SERIALIZABLE_CLASS); + + final NormalizedNode actualNode = readStore(shard, TestModel.OUTER_LIST_PATH); + assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testReadyLocalTransactionWithThreePhaseCommit() throws Exception{ + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testReadyLocalTransactionWithThreePhaseCommit"); + + waitUntilLeader(shard); + + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); + + final DataTreeModification modification = dataStore.getDataTree().takeSnapshot().newModification(); + + final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + new WriteModification(TestModel.TEST_PATH, writeData).apply(modification); + final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(); + new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification); + + final String txId = "tx1"; + modification.ready(); + final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, false); + + shard.tell(readyMessage, getRef()); + + expectMsgClass(ReadyTransactionReply.class); + + // Send the CanCommitTransaction message. + + shard.tell(new CanCommitTransaction(txId).toSerializable(), getRef()); + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + expectMsgClass(CanCommitTransactionReply.SERIALIZABLE_CLASS)); + assertEquals("Can commit", true, canCommitReply.getCanCommit()); + + // Send the CanCommitTransaction message. + + shard.tell(new CommitTransaction(txId).toSerializable(), getRef()); + expectMsgClass(CommitTransactionReply.SERIALIZABLE_CLASS); + + final NormalizedNode actualNode = readStore(shard, TestModel.OUTER_LIST_PATH); + assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testCommitWithPersistenceDisabled() throws Throwable { + dataStoreContextBuilder.persistent(false); + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCommitWithPersistenceDisabled"); + + waitUntilLeader(shard); + + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); + + // Setup a simulated transactions with a mock cohort. + + final String transactionID = "tx"; + final MutableCompositeModification modification = new MutableCompositeModification(); + final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore, + TestModel.TEST_PATH, containerNode, modification); + + final FiniteDuration duration = duration("5 seconds"); + + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. + + shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, + cohort, modification, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + // Send the CanCommitTransaction message. + + shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); + assertEquals("Can commit", true, canCommitReply.getCanCommit()); // Send the CanCommitTransaction message. shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef()); expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); - NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); + final InOrder inOrder = inOrder(cohort); + inOrder.verify(cohort).canCommit(); + inOrder.verify(cohort).preCommit(); + inOrder.verify(cohort).commit(); + + final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; } + private 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(CARS_QNAME))).when(mockCandidateNode).getDataAfter(); + doReturn(YangInstanceIdentifier.builder().build()).when(mockCandidate).getRootPath(); + doReturn(mockCandidateNode).when(mockCandidate).getRootNode(); + return mockCandidate; + } + + private 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.builder().build()).when(mockCandidate).getRootPath(); + doReturn(mockCandidateNode).when(mockCandidate).getRootNode(); + return mockCandidate; + } + @Test public void testCommitWhenTransactionHasNoModifications(){ // Note that persistence is enabled which would normally result in the entry getting written to the journal @@ -851,43 +1337,47 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - String transactionID = "tx1"; - MutableCompositeModification modification = new MutableCompositeModification(); - DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1"); + final String transactionID = "tx1"; + final MutableCompositeModification modification = new MutableCompositeModification(); + final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).commit(); + doReturn(mockUnmodifiedCandidate("cohort1-candidate")).when(cohort).getCandidate(); - FiniteDuration duration = duration("5 seconds"); + final FiniteDuration duration = duration("5 seconds"); // Simulate the ForwardedReadyTransaction messages that would be sent // by the ShardTransaction. shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, - cohort, modification, true), getRef()); - expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS); + cohort, modification, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message. shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef()); expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class); - InOrder inOrder = inOrder(cohort); + final InOrder inOrder = inOrder(cohort); inOrder.verify(cohort).canCommit(); inOrder.verify(cohort).preCommit(); inOrder.verify(cohort).commit(); + shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef()); + final ShardStats shardStats = expectMsgClass(duration, ShardStats.class); + // Use MBean for verification // Committed transaction count should increase as usual - assertEquals(1,shard.underlyingActor().getShardMBean().getCommittedTransactionsCount()); + assertEquals(1,shardStats.getCommittedTransactionsCount()); // Commit index should not advance because this does not go into the journal - assertEquals(-1, shard.underlyingActor().getShardMBean().getCommitIndex()); + assertEquals(-1, shardStats.getCommitIndex()); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); @@ -905,44 +1395,48 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - String transactionID = "tx1"; - MutableCompositeModification modification = new MutableCompositeModification(); + final String transactionID = "tx1"; + final MutableCompositeModification modification = new MutableCompositeModification(); modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build())); - DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1"); + final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).commit(); + doReturn(mockCandidate("cohort1-candidate")).when(cohort).getCandidate(); - FiniteDuration duration = duration("5 seconds"); + final FiniteDuration duration = duration("5 seconds"); // Simulate the ForwardedReadyTransaction messages that would be sent // by the ShardTransaction. shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, - cohort, modification, true), getRef()); - expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS); + cohort, modification, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message. shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef()); expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class); - InOrder inOrder = inOrder(cohort); + final InOrder inOrder = inOrder(cohort); inOrder.verify(cohort).canCommit(); inOrder.verify(cohort).preCommit(); inOrder.verify(cohort).commit(); + shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef()); + final ShardStats shardStats = expectMsgClass(duration, ShardStats.class); + // Use MBean for verification // Committed transaction count should increase as usual - assertEquals(1, shard.underlyingActor().getShardMBean().getCommittedTransactionsCount()); + assertEquals(1, shardStats.getCommittedTransactionsCount()); // Commit index should advance as we do not have an empty modification - assertEquals(0, shard.underlyingActor().getShardMBean().getCommitIndex()); + assertEquals(0, shardStats.getCommitIndex()); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); @@ -959,52 +1453,47 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - // Setup 2 mock cohorts. The first one fails in the commit phase. + // Setup 2 simulated transactions with mock cohorts. The first one fails in the + // commit phase. final String transactionID1 = "tx1"; - final DOMStoreThreePhaseCommitCohort cohort1 = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1"); + final MutableCompositeModification modification1 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit(); doReturn(Futures.immediateFuture(null)).when(cohort1).preCommit(); doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort1).commit(); + doReturn(mockCandidate("cohort1-candidate")).when(cohort1).getCandidate(); final String transactionID2 = "tx2"; - final DOMStoreThreePhaseCommitCohort cohort2 = mock(DOMStoreThreePhaseCommitCohort.class, "cohort2"); + final MutableCompositeModification modification2 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); - ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() { - @Override - public DOMStoreThreePhaseCommitCohort decorate(String transactionID, - DOMStoreThreePhaseCommitCohort actual) { - return transactionID1.equals(transactionID) ? cohort1 : cohort2; - } - }; - - shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator); - - FiniteDuration duration = duration("5 seconds"); + final FiniteDuration duration = duration("5 seconds"); final Timeout timeout = new Timeout(duration); - // Send BatchedModifications to start and ready each transaction. + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. - shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort1, modification1, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort2, modification2, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message for the first Tx. shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and // processed after the first Tx completes. - Future canCommitFuture = Patterns.ask(shard, + final Future canCommitFuture = Patterns.ask(shard, new CanCommitTransaction(transactionID2).toSerializable(), timeout); // Send the CommitTransaction message for the first Tx. This should send back an error @@ -1025,7 +1514,7 @@ public class ShardTest extends AbstractShardTest { assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS)); - InOrder inOrder = inOrder(cohort1, cohort2); + final InOrder inOrder = inOrder(cohort1, cohort2); inOrder.verify(cohort1).canCommit(); inOrder.verify(cohort1).preCommit(); inOrder.verify(cohort1).commit(); @@ -1044,45 +1533,66 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - String transactionID = "tx1"; - final DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1"); - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); - doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).preCommit(); + final String transactionID1 = "tx1"; + final MutableCompositeModification modification1 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1"); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit(); + doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort1).preCommit(); - ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() { - @Override - public DOMStoreThreePhaseCommitCohort decorate(String transactionID, - DOMStoreThreePhaseCommitCohort actual) { - return cohort; - } - }; + final String transactionID2 = "tx2"; + final MutableCompositeModification modification2 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); - shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator); + final FiniteDuration duration = duration("5 seconds"); + final Timeout timeout = new Timeout(duration); - FiniteDuration duration = duration("5 seconds"); + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. - // Send BatchedModifications to start and ready a transaction. + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort1, modification1, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort2, modification2, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); - // Send the CanCommitTransaction message. + // Send the CanCommitTransaction message for the first Tx. - shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); - // Send the CommitTransaction message. This should send back an error - // for preCommit failure. + // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and + // processed after the first Tx completes. + + final Future canCommitFuture = Patterns.ask(shard, + new CanCommitTransaction(transactionID2).toSerializable(), timeout); - shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef()); + // Send the CommitTransaction message for the first Tx. This should send back an error + // and trigger the 2nd Tx to proceed. + + shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef()); expectMsgClass(duration, akka.actor.Status.Failure.class); - InOrder inOrder = inOrder(cohort); - inOrder.verify(cohort).canCommit(); - inOrder.verify(cohort).preCommit(); + // Wait for the 2nd Tx to complete the canCommit phase. + + final CountDownLatch latch = new CountDownLatch(1); + canCommitFuture.onComplete(new OnComplete() { + @Override + public void onComplete(final Throwable t, final Object resp) { + latch.countDown(); + } + }, getSystem().dispatcher()); + + assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS)); + + final InOrder inOrder = inOrder(cohort1, cohort2); + inOrder.verify(cohort1).canCommit(); + inOrder.verify(cohort1).preCommit(); + inOrder.verify(cohort2).canCommit(); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; @@ -1099,31 +1609,183 @@ public class ShardTest extends AbstractShardTest { final FiniteDuration duration = duration("5 seconds"); - String transactionID = "tx1"; - final DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1"); + final String transactionID1 = "tx1"; + final MutableCompositeModification modification = new MutableCompositeModification(); + final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit(); - ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() { - @Override - public DOMStoreThreePhaseCommitCohort decorate(String transactionID, - DOMStoreThreePhaseCommitCohort actual) { - return cohort; - } - }; + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. - shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator); + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort, modification, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); - // Send BatchedModifications to start and ready a transaction. + // Send the CanCommitTransaction message. - shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); + expectMsgClass(duration, akka.actor.Status.Failure.class); + + // Send another can commit to ensure the failed one got cleaned up. + + reset(cohort); + + final String transactionID2 = "tx2"; + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); + + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort, modification, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef()); + final CanCommitTransactionReply reply = CanCommitTransactionReply.fromSerializable( + expectMsgClass(CanCommitTransactionReply.SERIALIZABLE_CLASS)); + assertEquals("getCanCommit", true, reply.getCanCommit()); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testCanCommitPhaseFalseResponse() throws Throwable { + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCanCommitPhaseFalseResponse"); + + waitUntilLeader(shard); + + final FiniteDuration duration = duration("5 seconds"); + + final String transactionID1 = "tx1"; + final MutableCompositeModification modification = new MutableCompositeModification(); + final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(cohort).canCommit(); + + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. + + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort, modification, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message. - shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); + shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); + CanCommitTransactionReply reply = CanCommitTransactionReply.fromSerializable( + expectMsgClass(CanCommitTransactionReply.SERIALIZABLE_CLASS)); + assertEquals("getCanCommit", false, reply.getCanCommit()); + + // Send another can commit to ensure the failed one got cleaned up. + + reset(cohort); + + final String transactionID2 = "tx2"; + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); + + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort, modification, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef()); + reply = CanCommitTransactionReply.fromSerializable( + expectMsgClass(CanCommitTransactionReply.SERIALIZABLE_CLASS)); + assertEquals("getCanCommit", true, reply.getCanCommit()); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testImmediateCommitWithCanCommitPhaseFailure() throws Throwable { + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testImmediateCommitWithCanCommitPhaseFailure"); + + waitUntilLeader(shard); + + final FiniteDuration duration = duration("5 seconds"); + + final String transactionID1 = "tx1"; + final MutableCompositeModification modification = new MutableCompositeModification(); + final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); + doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit(); + + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. + + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort, modification, true, true), getRef()); + + expectMsgClass(duration, akka.actor.Status.Failure.class); + + // Send another can commit to ensure the failed one got cleaned up. + + reset(cohort); + + final String transactionID2 = "tx2"; + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); + doReturn(Futures.immediateFuture(null)).when(cohort).preCommit(); + doReturn(Futures.immediateFuture(null)).when(cohort).commit(); + final DataTreeCandidateTip candidate = mock(DataTreeCandidateTip.class); + final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class); + doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType(); + doReturn(candidateRoot).when(candidate).getRootNode(); + doReturn(candidate).when(cohort).getCandidate(); + + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort, modification, true, true), getRef()); + + expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testImmediateCommitWithCanCommitPhaseFalseResponse() throws Throwable { + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testImmediateCommitWithCanCommitPhaseFalseResponse"); + + waitUntilLeader(shard); + + final FiniteDuration duration = duration("5 seconds"); + + final String transactionID = "tx1"; + final MutableCompositeModification modification = new MutableCompositeModification(); + final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(cohort).canCommit(); + + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. + + shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, + cohort, modification, true, true), getRef()); + expectMsgClass(duration, akka.actor.Status.Failure.class); + // Send another can commit to ensure the failed one got cleaned up. + + reset(cohort); + + final String transactionID2 = "tx2"; + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); + doReturn(Futures.immediateFuture(null)).when(cohort).preCommit(); + doReturn(Futures.immediateFuture(null)).when(cohort).commit(); + final DataTreeCandidateTip candidate = mock(DataTreeCandidateTip.class); + final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class); + doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType(); + doReturn(candidateRoot).when(candidate).getRootNode(); + doReturn(candidate).when(cohort).getCandidate(); + + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort, modification, true, true), getRef()); + + expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; } @@ -1138,14 +1800,14 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); final FiniteDuration duration = duration("5 seconds"); - InMemoryDOMDataStore dataStore = shard.underlyingActor().getDataStore(); + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); final String transactionID = "tx1"; - Function> preCommit = - new Function>() { + final Function> preCommit = + new Function>() { @Override - public ListenableFuture apply(final DOMStoreThreePhaseCommitCohort cohort) { - ListenableFuture preCommitFuture = cohort.preCommit(); + public ListenableFuture apply(final ShardDataTreeCohort cohort) { + final ListenableFuture preCommitFuture = cohort.preCommit(); // Simulate an AbortTransaction message occurring during replication, after // persisting and before finishing the commit to the in-memory store. @@ -1161,19 +1823,24 @@ public class ShardTest extends AbstractShardTest { } }; - shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + final MutableCompositeModification modification = new MutableCompositeModification(); + final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort1", dataStore, + TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), + modification, preCommit); + + shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, + cohort, modification, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef()); expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); - NormalizedNode node = readStore(shard, TestModel.TEST_PATH); + final NormalizedNode node = readStore(shard, TestModel.TEST_PATH); // Since we're simulating an abort occurring during replication and before finish commit, // the data should still get written to the in-memory store since we've gotten past @@ -1197,26 +1864,42 @@ public class ShardTest extends AbstractShardTest { final FiniteDuration duration = duration("5 seconds"); + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); + writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); writeToStore(shard, TestModel.OUTER_LIST_PATH, ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build()); - // Create and ready the 1st Tx - will timeout - - String transactionID1 = "tx1"; - shard.tell(newBatchedModifications(transactionID1, YangInstanceIdentifier.builder( - TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(), - ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); - - // Create and ready the 2nd Tx + // Create 1st Tx - will timeout - String transactionID2 = "tx2"; - YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) - .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(); - shard.tell(newBatchedModifications(transactionID2, listNodePath, - ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + final String transactionID1 = "tx1"; + final MutableCompositeModification modification1 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, + YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) + .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(), + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), + modification1); + + // Create 2nd Tx + + final String transactionID2 = "tx3"; + final MutableCompositeModification modification2 = new MutableCompositeModification(); + final YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) + .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(); + final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort3", dataStore, + listNodePath, + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2), + modification2); + + // Ready the Tx's + + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort1, modification1, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort2, modification2, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // canCommit 1st Tx. We don't send the commit so it should timeout. @@ -1228,12 +1911,17 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef()); expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS); + // Try to commit the 1st Tx - should fail as it's not the current Tx. + + shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef()); + expectMsgClass(duration, akka.actor.Status.Failure.class); + // Commit the 2nd Tx. shard.tell(new CommitTransaction(transactionID2).toSerializable(), getRef()); expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); - NormalizedNode node = readStore(shard, listNodePath); + final NormalizedNode node = readStore(shard, listNodePath); assertNotNull(listNodePath + " not found", node); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); @@ -1242,7 +1930,7 @@ public class ShardTest extends AbstractShardTest { @Test public void testTransactionCommitQueueCapacityExceeded() throws Throwable { - dataStoreContextBuilder.shardTransactionCommitQueueCapacity(1); + dataStoreContextBuilder.shardTransactionCommitQueueCapacity(2); new ShardTestKit(getSystem()) {{ final TestActorRef shard = TestActorRef.create(getSystem(), @@ -1253,23 +1941,40 @@ public class ShardTest extends AbstractShardTest { final FiniteDuration duration = duration("5 seconds"); - String transactionID1 = "tx1"; - String transactionID2 = "tx2"; - String transactionID3 = "tx3"; + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - // Send a BatchedModifications to start transactions and ready them. + final String transactionID1 = "tx1"; + final MutableCompositeModification modification1 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, + TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1); - shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + final String transactionID2 = "tx2"; + final MutableCompositeModification modification2 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore, + TestModel.OUTER_LIST_PATH, + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), + modification2); - shard.tell(newBatchedModifications(transactionID2,TestModel.OUTER_LIST_PATH, - ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + final String transactionID3 = "tx3"; + final MutableCompositeModification modification3 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore, + TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification3); - shard.tell(newBatchedModifications(transactionID3, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + // Ready the Tx's + + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort1, modification1, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort2, modification2, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + // The 3rd Tx should exceed queue capacity and fail. + + shard.tell(new ForwardedReadyTransaction(transactionID3, CURRENT_VERSION, + cohort3, modification3, true, false), getRef()); + expectMsgClass(duration, akka.actor.Status.Failure.class); // canCommit 1st Tx. @@ -1289,6 +1994,126 @@ public class ShardTest extends AbstractShardTest { }}; } + @Test + public void testTransactionCommitWithPriorExpiredCohortEntries() throws Throwable { + dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1); + + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testTransactionCommitWithPriorExpiredCohortEntries"); + + waitUntilLeader(shard); + + final FiniteDuration duration = duration("5 seconds"); + + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); + + final String transactionID1 = "tx1"; + final MutableCompositeModification modification1 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, + TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1); + + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort1, modification1, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + final String transactionID2 = "tx2"; + final MutableCompositeModification modification2 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore, + TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2); + + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort2, modification2, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + final String transactionID3 = "tx3"; + final MutableCompositeModification modification3 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore, + TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME), modification3); + + shard.tell(new ForwardedReadyTransaction(transactionID3, CURRENT_VERSION, + cohort3, modification3, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + // All Tx's are readied. We'll send canCommit for the last one but not the others. The others + // should expire from the queue and the last one should be processed. + + shard.tell(new CanCommitTransaction(transactionID3).toSerializable(), getRef()); + expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testTransactionCommitWithSubsequentExpiredCohortEntry() throws Throwable { + dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1); + + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testTransactionCommitWithSubsequentExpiredCohortEntry"); + + waitUntilLeader(shard); + + final FiniteDuration duration = duration("5 seconds"); + + final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); + + final String transactionID1 = "tx1"; + final MutableCompositeModification modification1 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, + TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1); + + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort1, modification1, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + // CanCommit the first one so it's the current in-progress CohortEntry. + + shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); + expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS); + + // Ready the second Tx. + + final String transactionID2 = "tx2"; + final MutableCompositeModification modification2 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore, + TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2); + + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort2, modification2, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + // Ready the third Tx. + + final String transactionID3 = "tx3"; + final DataTreeModification modification3 = dataStore.getDataTree().takeSnapshot().newModification(); + new WriteModification(TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME)) + .apply(modification3); + modification3.ready(); + final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(transactionID3, modification3, true); + + shard.tell(readyMessage, getRef()); + + // Commit the first Tx. After completing, the second should expire from the queue and the third + // Tx committed. + + shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef()); + expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); + + // Expect commit reply from the third Tx. + + expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); + + final NormalizedNode node = readStore(shard, TestModel.TEST2_PATH); + assertNotNull(TestModel.TEST2_PATH + " not found", node); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + @Test public void testCanCommitBeforeReadyFailure() throws Throwable { new ShardTestKit(getSystem()) {{ @@ -1315,48 +2140,41 @@ public class ShardTest extends AbstractShardTest { // Setup 2 simulated transactions with mock cohorts. The first one will be aborted. final String transactionID1 = "tx1"; - final DOMStoreThreePhaseCommitCohort cohort1 = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1"); + final MutableCompositeModification modification1 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit(); doReturn(Futures.immediateFuture(null)).when(cohort1).abort(); final String transactionID2 = "tx2"; - final DOMStoreThreePhaseCommitCohort cohort2 = mock(DOMStoreThreePhaseCommitCohort.class, "cohort2"); + final MutableCompositeModification modification2 = new MutableCompositeModification(); + final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); - FiniteDuration duration = duration("5 seconds"); + final FiniteDuration duration = duration("5 seconds"); final Timeout timeout = new Timeout(duration); - ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() { - @Override - public DOMStoreThreePhaseCommitCohort decorate(String transactionID, - DOMStoreThreePhaseCommitCohort actual) { - return transactionID1.equals(transactionID) ? cohort1 : cohort2; - } - }; + // Simulate the ForwardedReadyTransaction messages that would be sent + // by the ShardTransaction. - shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator); - - // Send BatchedModifications to start and ready each transaction. - - shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, + cohort1, modification1, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, - ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); - expectMsgClass(duration, BatchedModificationsReply.class); + shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, + cohort2, modification2, true, false), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message for the first Tx. shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); - CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and // processed after the first Tx completes. - Future canCommitFuture = Patterns.ask(shard, + final Future canCommitFuture = Patterns.ask(shard, new CanCommitTransaction(transactionID2).toSerializable(), timeout); // Send the AbortTransaction message for the first Tx. This should trigger the 2nd @@ -1369,7 +2187,7 @@ public class ShardTest extends AbstractShardTest { Await.ready(canCommitFuture, duration); - InOrder inOrder = inOrder(cohort1, cohort2); + final InOrder inOrder = inOrder(cohort1, cohort2); inOrder.verify(cohort1).canCommit(); inOrder.verify(cohort2).canCommit(); @@ -1390,68 +2208,39 @@ public class ShardTest extends AbstractShardTest { @SuppressWarnings("serial") public void testCreateSnapshot(final boolean persistent, final String shardActorName) throws Exception{ - final AtomicReference savedSnapshot = new AtomicReference<>(); - class DelegatingPersistentDataProvider implements DataPersistenceProvider { - DataPersistenceProvider delegate; - - DelegatingPersistentDataProvider(DataPersistenceProvider delegate) { - this.delegate = delegate; - } - - @Override - public boolean isRecoveryApplicable() { - return delegate.isRecoveryApplicable(); - } + final AtomicReference latch = new AtomicReference<>(new CountDownLatch(1)); - @Override - public void persist(T o, Procedure procedure) { - delegate.persist(o, procedure); + final AtomicReference savedSnapshot = new AtomicReference<>(); + class TestPersistentDataProvider extends DelegatingPersistentDataProvider { + TestPersistentDataProvider(final DataPersistenceProvider delegate) { + super(delegate); } @Override - public void saveSnapshot(Object o) { + public void saveSnapshot(final Object o) { savedSnapshot.set(o); - delegate.saveSnapshot(o); - } - - @Override - public void deleteSnapshots(SnapshotSelectionCriteria criteria) { - delegate.deleteSnapshots(criteria); - } - - @Override - public void deleteMessages(long sequenceNumber) { - delegate.deleteMessages(sequenceNumber); + super.saveSnapshot(o); } } dataStoreContextBuilder.persistent(persistent); - - new ShardTestKit(getSystem()) {{ - final AtomicReference latch = new AtomicReference<>(new CountDownLatch(1)); - class TestShard extends Shard { - protected TestShard(ShardIdentifier name, Map peerAddresses, - DatastoreContext datastoreContext, SchemaContext schemaContext) { + protected TestShard(final ShardIdentifier name, final Map peerAddresses, + final DatastoreContext datastoreContext, final SchemaContext schemaContext) { super(name, peerAddresses, datastoreContext, schemaContext); + setPersistence(new TestPersistentDataProvider(super.persistence())); } - DelegatingPersistentDataProvider delegating; + @Override + public void handleCommand(final Object message) { + super.handleCommand(message); - protected DataPersistenceProvider persistence() { - if(delegating == null) { - delegating = new DelegatingPersistentDataProvider(super.persistence()); + if (message instanceof SaveSnapshotSuccess || message.equals("commit_snapshot")) { + latch.get().countDown(); } - return delegating; - } - - @Override - protected void commitSnapshot(final long sequenceNumber) { - super.commitSnapshot(sequenceNumber); - latch.get().countDown(); } @Override @@ -1460,7 +2249,7 @@ public class ShardTest extends AbstractShardTest { } } - Creator creator = new Creator() { + final Creator creator = new Creator() { @Override public Shard create() throws Exception { return new TestShard(shardID, Collections.emptyMap(), @@ -1468,17 +2257,17 @@ public class ShardTest extends AbstractShardTest { } }; - TestActorRef shard = TestActorRef.create(getSystem(), + final TestActorRef shard = TestActorRef.create(getSystem(), Props.create(new DelegatingShardCreator(creator)), shardActorName); waitUntilLeader(shard); writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build()); + final NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build()); // Trigger creation of a snapshot by ensuring - RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext(); + final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext(); raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1); assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS)); @@ -1503,9 +2292,9 @@ public class ShardTest extends AbstractShardTest { shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); } - private void verifySnapshot(Snapshot snapshot, NormalizedNode expectedRoot) { + private void verifySnapshot(final Snapshot snapshot, final NormalizedNode expectedRoot) { - NormalizedNode actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState()); + final NormalizedNode actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState()); assertEquals("Root node", expectedRoot, actual); }}; @@ -1514,29 +2303,29 @@ public class ShardTest extends AbstractShardTest { /** * This test simply verifies that the applySnapShot logic will work * @throws ReadFailedException + * @throws DataValidationFailedException */ @Test - public void testInMemoryDataStoreRestore() throws ReadFailedException { - InMemoryDOMDataStore store = new InMemoryDOMDataStore("test", MoreExecutors.sameThreadExecutor()); - - store.onGlobalContextUpdated(SCHEMA_CONTEXT); + public void testInMemoryDataTreeRestore() throws ReadFailedException, DataValidationFailedException { + final DataTree store = InMemoryDataTreeFactory.getInstance().create(); + store.setSchemaContext(SCHEMA_CONTEXT); - DOMStoreWriteTransaction putTransaction = store.newWriteOnlyTransaction(); + final DataTreeModification putTransaction = store.takeSnapshot().newModification(); putTransaction.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - commitTransaction(putTransaction); + commitTransaction(store, putTransaction); - NormalizedNode expected = readStore(store); + final NormalizedNode expected = readStore(store, YangInstanceIdentifier.builder().build()); - DOMStoreWriteTransaction writeTransaction = store.newWriteOnlyTransaction(); + final DataTreeModification writeTransaction = store.takeSnapshot().newModification(); writeTransaction.delete(YangInstanceIdentifier.builder().build()); writeTransaction.write(YangInstanceIdentifier.builder().build(), expected); - commitTransaction(writeTransaction); + commitTransaction(store, writeTransaction); - NormalizedNode actual = readStore(store); + final NormalizedNode actual = readStore(store, YangInstanceIdentifier.builder().build()); assertEquals(expected, actual); } @@ -1557,17 +2346,17 @@ public class ShardTest extends AbstractShardTest { nonPersistentContext, SCHEMA_CONTEXT); new ShardTestKit(getSystem()) {{ - TestActorRef shard1 = TestActorRef.create(getSystem(), + final TestActorRef shard1 = TestActorRef.create(getSystem(), persistentProps, "testPersistence1"); - assertTrue("Recovery Applicable", shard1.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable()); + assertTrue("Recovery Applicable", shard1.underlyingActor().persistence().isRecoveryApplicable()); shard1.tell(PoisonPill.getInstance(), ActorRef.noSender()); - TestActorRef shard2 = TestActorRef.create(getSystem(), + final TestActorRef shard2 = TestActorRef.create(getSystem(), nonPersistentProps, "testPersistence2"); - assertFalse("Recovery Not Applicable", shard2.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable()); + assertFalse("Recovery Not Applicable", shard2.underlyingActor().persistence().isRecoveryApplicable()); shard2.tell(PoisonPill.getInstance(), ActorRef.noSender()); @@ -1580,22 +2369,22 @@ public class ShardTest extends AbstractShardTest { new ShardTestKit(getSystem()) {{ dataStoreContextBuilder.persistent(true); - TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), "testOnDatastoreContext"); + final TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), "testOnDatastoreContext"); assertEquals("isRecoveryApplicable", true, - shard.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable()); + shard.underlyingActor().persistence().isRecoveryApplicable()); waitUntilLeader(shard); shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender()); assertEquals("isRecoveryApplicable", false, - shard.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable()); + shard.underlyingActor().persistence().isRecoveryApplicable()); shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender()); assertEquals("isRecoveryApplicable", true, - shard.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable()); + shard.underlyingActor().persistence().isRecoveryApplicable()); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; @@ -1611,19 +2400,32 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - TestActorRef listener = + final TestActorRef listener = TestActorRef.create(getSystem(), Props.create(MessageCollectorActor.class)); shard.tell(new RegisterRoleChangeListener(), listener); - // TODO: MessageCollectorActor exists as a test util in both the akka-raft and distributed-datastore - // projects. Need to move it to commons as a regular utility and then we can get rid of this arbitrary - // sleep. - Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); + MessageCollectorActor.expectFirstMatching(listener, RegisterRoleChangeListenerReply.class); + + ShardLeaderStateChanged leaderStateChanged = MessageCollectorActor.expectFirstMatching(listener, + ShardLeaderStateChanged.class); + assertEquals("getLocalShardDataTree present", true, + leaderStateChanged.getLocalShardDataTree().isPresent()); + assertSame("getLocalShardDataTree", shard.underlyingActor().getDataStore().getDataTree(), + leaderStateChanged.getLocalShardDataTree().get()); - List allMatching = MessageCollectorActor.getAllMatching(listener, RegisterRoleChangeListenerReply.class); + MessageCollectorActor.clearMessages(listener); - assertEquals(1, allMatching.size()); + // Force a leader change + + shard.tell(new RequestVote(10000, "member2", 50, 50), getRef()); + + leaderStateChanged = MessageCollectorActor.expectFirstMatching(listener, + ShardLeaderStateChanged.class); + assertEquals("getLocalShardDataTree present", false, + leaderStateChanged.getLocalShardDataTree().isPresent()); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); } }; } @@ -1645,15 +2447,9 @@ public class ShardTest extends AbstractShardTest { shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); } - private void commitTransaction(final DOMStoreWriteTransaction transaction) { - DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready(); - ListenableFuture future = - commitCohort.preCommit(); - try { - future.get(); - future = commitCohort.commit(); - future.get(); - } catch (InterruptedException | ExecutionException e) { - } + private static void commitTransaction(final DataTree store, final DataTreeModification modification) throws DataValidationFailedException { + modification.ready(); + store.validate(modification); + store.commit(store.prepare(modification)); } }