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=a87000136fc52729d4e760dc2ffa86af84346392;hp=2792342ab2f3921f451999651477edca5664397b;hb=1e884647502a8d91f8a57bde8193c60b9bbcce0d;hpb=81b58527159d1ae07e7042372ca78b7dc769a7c7 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 2792342ab2..a87000136f 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 @@ -5,24 +5,23 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; -import static org.opendaylight.controller.cluster.datastore.messages.CreateTransaction.CURRENT_VERSION; +import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION; import akka.actor.ActorRef; import akka.actor.PoisonPill; import akka.actor.Props; import akka.dispatch.Dispatchers; import akka.dispatch.OnComplete; import akka.japi.Creator; +import akka.japi.Procedure; import akka.pattern.Patterns; +import akka.persistence.SnapshotSelectionCriteria; 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.CheckedFuture; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; @@ -30,19 +29,16 @@ 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.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import org.mockito.InOrder; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; +import org.opendaylight.controller.cluster.DataPersistenceProvider; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction; import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply; @@ -57,38 +53,38 @@ import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionRe import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener; import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply; import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext; +import org.opendaylight.controller.cluster.datastore.modification.DeleteModification; import org.opendaylight.controller.cluster.datastore.modification.MergeModification; 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.node.NormalizedNodeToNodeCodec; -import org.opendaylight.controller.cluster.datastore.utils.InMemoryJournal; -import org.opendaylight.controller.cluster.datastore.utils.InMemorySnapshotStore; +import org.opendaylight.controller.cluster.datastore.utils.MessageCollectorActor; import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener; +import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils; +import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener; +import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListenerReply; 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.ApplyLogEntries; +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.CaptureSnapshot; 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.protobuff.client.messages.CompositeModificationByteStringPayload; -import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload; -import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload; +import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal; +import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore; 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.AsyncDataChangeEvent; -import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener; 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.common.NormalizedNodeMessages; +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.DOMStoreReadTransaction; import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort; import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; @@ -97,41 +93,11 @@ import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild; import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode; import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode; import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes; -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 AbstractActorTest { - - private static final SchemaContext SCHEMA_CONTEXT = TestModel.createTestContext(); - - private static final AtomicInteger NEXT_SHARD_NUM = new AtomicInteger(); - - private final ShardIdentifier shardID = ShardIdentifier.builder().memberName("member-1") - .shardName("inventory").type("config" + NEXT_SHARD_NUM.getAndIncrement()).build(); - - private DatastoreContext dataStoreContext = DatastoreContext.newBuilder(). - shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000). - shardHeartbeatIntervalInMillis(100).build(); - - @Before - public void setUp() { - InMemorySnapshotStore.clear(); - InMemoryJournal.clear(); - } - - @After - public void tearDown() { - InMemorySnapshotStore.clear(); - InMemoryJournal.clear(); - } - - private Props newShardProps() { - return Shard.props(shardID, Collections.emptyMap(), - dataStoreContext, SCHEMA_CONTEXT); - } - +public class ShardTest extends AbstractShardTest { @Test public void testRegisterChangeListener() throws Exception { new ShardTestKit(getSystem()) {{ @@ -186,7 +152,7 @@ public class ShardTest extends AbstractActorTest { @Override public Shard create() throws Exception { return new Shard(shardID, Collections.emptyMap(), - dataStoreContext, SCHEMA_CONTEXT) { + newDatastoreContext(), SCHEMA_CONTEXT) { @Override public void onReceiveCommand(final Object message) throws Exception { if(message instanceof ElectionTimeout && firstElectionTimeout) { @@ -314,7 +280,7 @@ public class ShardTest extends AbstractActorTest { class TestShard extends Shard { TestShard() { super(shardID, Collections.singletonMap(shardID, null), - dataStoreContext, SCHEMA_CONTEXT); + newDatastoreContext(), SCHEMA_CONTEXT); } Map getPeerAddresses() { @@ -358,25 +324,23 @@ public class ShardTest extends AbstractActorTest { TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), "testApplySnapshot"); - NormalizedNodeToNodeCodec codec = - new NormalizedNodeToNodeCodec(SCHEMA_CONTEXT); + InMemoryDOMDataStore store = new InMemoryDOMDataStore("OPER", MoreExecutors.sameThreadExecutor()); + store.onGlobalContextUpdated(SCHEMA_CONTEXT); - writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + writeToStore(store, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); YangInstanceIdentifier root = YangInstanceIdentifier.builder().build(); - NormalizedNode expected = readStore(shard, root); - - NormalizedNodeMessages.Container encode = codec.encode(expected); + NormalizedNode expected = readStore(store, root); ApplySnapshot applySnapshot = new ApplySnapshot(Snapshot.create( - encode.getNormalizedNode().toByteString().toByteArray(), + SerializationUtils.serializeNormalizedNode(expected), Collections.emptyList(), 1, 2, 3, 4)); shard.underlyingActor().onReceiveCommand(applySnapshot); NormalizedNode actual = readStore(shard, root); - assertEquals(expected, actual); + assertEquals("Root node", expected, actual); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); } @@ -388,11 +352,8 @@ public class ShardTest extends AbstractActorTest { NormalizedNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME); - MutableCompositeModification compMod = new MutableCompositeModification(); - compMod.addModification(new WriteModification(TestModel.TEST_PATH, node, SCHEMA_CONTEXT)); - Payload payload = new CompositeModificationPayload(compMod.toSerializable()); - ApplyState applyState = new ApplyState(null, "test", - new ReplicatedLogImplEntry(1, 2, payload)); + ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2, + newModificationPayload(new WriteModification(TestModel.TEST_PATH, node)))); shard.underlyingActor().onReceiveCommand(applyState); @@ -402,7 +363,6 @@ public class ShardTest extends AbstractActorTest { shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); } - @SuppressWarnings("serial") @Test public void testRecovery() throws Exception { @@ -411,186 +371,47 @@ public class ShardTest extends AbstractActorTest { InMemoryDOMDataStore testStore = InMemoryDOMDataStoreFactory.create("Test", null, null); testStore.onGlobalContextUpdated(SCHEMA_CONTEXT); - DOMStoreWriteTransaction writeTx = testStore.newWriteOnlyTransaction(); - writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - DOMStoreThreePhaseCommitCohort commitCohort = writeTx.ready(); - commitCohort.preCommit().get(); - commitCohort.commit().get(); + writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - DOMStoreReadTransaction readTx = testStore.newReadOnlyTransaction(); - NormalizedNode root = readTx.read(YangInstanceIdentifier.builder().build()).get().get(); + NormalizedNode root = readStore(testStore, YangInstanceIdentifier.builder().build()); InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create( - new NormalizedNodeToNodeCodec(SCHEMA_CONTEXT).encode( - root). - getNormalizedNode().toByteString().toByteArray(), - Collections.emptyList(), 0, 1, -1, -1)); + SerializationUtils.serializeNormalizedNode(root), + Collections.emptyList(), 0, 1, -1, -1)); // Set up the InMemoryJournal. - InMemoryJournal.addEntry(shardID.toString(), 0, new ReplicatedLogImplEntry(0, 1, newPayload( + InMemoryJournal.addEntry(shardID.toString(), 0, new ReplicatedLogImplEntry(0, 1, newModificationPayload( new WriteModification(TestModel.OUTER_LIST_PATH, - ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), - SCHEMA_CONTEXT)))); + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build())))); int nListEntries = 16; Set listEntryKeys = new HashSet<>(); - for(int i = 1; i <= nListEntries-5; i++) { - listEntryKeys.add(Integer.valueOf(i)); - YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) - .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build(); - Modification mod = new MergeModification(path, - ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i), - SCHEMA_CONTEXT); - InMemoryJournal.addEntry(shardID.toString(), i, new ReplicatedLogImplEntry(i, 1, - newPayload(mod))); - } - // Add some of the new CompositeModificationByteStringPayload - for(int i = 11; i <= nListEntries; i++) { + // Add some ModificationPayload entries + for(int i = 1; i <= nListEntries; i++) { listEntryKeys.add(Integer.valueOf(i)); YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build(); Modification mod = new MergeModification(path, - ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i), - SCHEMA_CONTEXT); + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i)); InMemoryJournal.addEntry(shardID.toString(), i, new ReplicatedLogImplEntry(i, 1, - newByteStringPayload(mod))); + newModificationPayload(mod))); } - InMemoryJournal.addEntry(shardID.toString(), nListEntries + 1, - new ApplyLogEntries(nListEntries)); - - // Create the actor and wait for recovery complete. - - final CountDownLatch recoveryComplete = new CountDownLatch(1); - - Creator creator = new Creator() { - @Override - public Shard create() throws Exception { - return new Shard(shardID, Collections.emptyMap(), - dataStoreContext, SCHEMA_CONTEXT) { - @Override - protected void onRecoveryComplete() { - try { - super.onRecoveryComplete(); - } finally { - recoveryComplete.countDown(); - } - } - }; - } - }; + new ApplyJournalEntries(nListEntries)); - TestActorRef shard = TestActorRef.create(getSystem(), - Props.create(new DelegatingShardCreator(creator)), "testRecovery"); - - assertEquals("Recovery complete", true, recoveryComplete.await(5, TimeUnit.SECONDS)); - - // 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); - for(Object entry: (Iterable) outerList.getValue()) { - 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()); - Object value = idLeaf.get().getValue(); - assertTrue("Unexpected value for leaf "+ TestModel.ID_QNAME.getLocalName() + ": " + value, - listEntryKeys.remove(value)); - } - - if(!listEntryKeys.isEmpty()) { - fail("Missing " + TestModel.OUTER_LIST_QNAME.getLocalName() + " entries with keys: " + - listEntryKeys); - } - - assertEquals("Last log index", nListEntries, - shard.underlyingActor().getShardMBean().getLastLogIndex()); - assertEquals("Commit index", nListEntries, - shard.underlyingActor().getShardMBean().getCommitIndex()); - assertEquals("Last applied", nListEntries, - shard.underlyingActor().getShardMBean().getLastApplied()); - - shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); - } - - private CompositeModificationPayload newPayload(final Modification... mods) { - MutableCompositeModification compMod = new MutableCompositeModification(); - for(Modification mod: mods) { - compMod.addModification(mod); - } - - return new CompositeModificationPayload(compMod.toSerializable()); + testRecovery(listEntryKeys); } - private CompositeModificationByteStringPayload newByteStringPayload(final Modification... mods) { + private ModificationPayload newModificationPayload(final Modification... mods) throws IOException { MutableCompositeModification compMod = new MutableCompositeModification(); for(Modification mod: mods) { compMod.addModification(mod); } - return new CompositeModificationByteStringPayload(compMod.toSerializable()); - } - - - private DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(final String cohortName, - final InMemoryDOMDataStore dataStore, final YangInstanceIdentifier path, final NormalizedNode data, - final MutableCompositeModification modification) { - return setupMockWriteTransaction(cohortName, dataStore, path, data, modification, null); - } - - private DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(final String cohortName, - final InMemoryDOMDataStore dataStore, final YangInstanceIdentifier path, final NormalizedNode data, - final MutableCompositeModification modification, - final Function> preCommit) { - - DOMStoreWriteTransaction tx = dataStore.newWriteOnlyTransaction(); - tx.write(path, data); - final DOMStoreThreePhaseCommitCohort realCohort = tx.ready(); - DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, cohortName); - - doAnswer(new Answer>() { - @Override - public ListenableFuture answer(final InvocationOnMock invocation) { - return realCohort.canCommit(); - } - }).when(cohort).canCommit(); - - doAnswer(new Answer>() { - @Override - public ListenableFuture answer(final InvocationOnMock invocation) throws Throwable { - if(preCommit != null) { - return preCommit.apply(realCohort); - } else { - return realCohort.preCommit(); - } - } - }).when(cohort).preCommit(); - - doAnswer(new Answer>() { - @Override - public ListenableFuture answer(final InvocationOnMock invocation) throws Throwable { - return realCohort.commit(); - } - }).when(cohort).commit(); - - doAnswer(new Answer>() { - @Override - public ListenableFuture answer(final InvocationOnMock invocation) throws Throwable { - return realCohort.abort(); - } - }).when(cohort).abort(); - - modification.addModification(new WriteModification(path, data, SCHEMA_CONTEXT)); - - return cohort; + return new ModificationPayload(compMod); } @SuppressWarnings({ "unchecked" }) @@ -774,20 +595,176 @@ public class ShardTest extends AbstractActorTest { assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent()); assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue()); - for(int i = 0; i < 20 * 5; i++) { - long lastLogIndex = shard.underlyingActor().getShardMBean().getLastLogIndex(); - if(lastLogIndex == 2) { - break; - } - Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS); - } + verifyLastLogIndex(shard, 2); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } - assertEquals("Last log index", 2, shard.underlyingActor().getShardMBean().getLastLogIndex()); + @Test + public void testCommitWithPersistenceDisabled() throws Throwable { + dataStoreContextBuilder.persistent(false); + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCommitPhaseFailure"); + + waitUntilLeader(shard); + + InMemoryDOMDataStore dataStore = shard.underlyingActor().getDataStore(); + + // Setup a simulated transactions with a mock cohort. + + String transactionID = "tx"; + MutableCompositeModification modification = new MutableCompositeModification(); + NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + DOMStoreThreePhaseCommitCohort cohort = setupMockWriteTransaction("cohort", dataStore, + TestModel.TEST_PATH, containerNode, modification); + + 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); + + // Send the CanCommitTransaction message. + + shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); + 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); + + InOrder inOrder = inOrder(cohort); + inOrder.verify(cohort).canCommit(); + inOrder.verify(cohort).preCommit(); + inOrder.verify(cohort).commit(); + + NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); + assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; } + @Test + public void testCommitWhenTransactionHasNoModifications(){ + // Note that persistence is enabled which would normally result in the entry getting written to the journal + // but here that need not happen + new ShardTestKit(getSystem()) { + { + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCommitWhenTransactionHasNoModifications"); + + waitUntilLeader(shard); + + String transactionID = "tx1"; + MutableCompositeModification modification = new MutableCompositeModification(); + DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.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(); + + 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); + + // Send the CanCommitTransaction message. + + 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 CommitTransaction(transactionID).toSerializable(), getRef()); + expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class); + + InOrder inOrder = inOrder(cohort); + inOrder.verify(cohort).canCommit(); + inOrder.verify(cohort).preCommit(); + inOrder.verify(cohort).commit(); + + // Use MBean for verification + // Committed transaction count should increase as usual + assertEquals(1,shard.underlyingActor().getShardMBean().getCommittedTransactionsCount()); + + // Commit index should not advance because this does not go into the journal + assertEquals(-1, shard.underlyingActor().getShardMBean().getCommitIndex()); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + + } + }; + } + + @Test + public void testCommitWhenTransactionHasModifications(){ + new ShardTestKit(getSystem()) { + { + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCommitWhenTransactionHasModifications"); + + waitUntilLeader(shard); + + String transactionID = "tx1"; + MutableCompositeModification modification = new MutableCompositeModification(); + modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build())); + DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.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(); + + 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); + + // Send the CanCommitTransaction message. + + 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 CommitTransaction(transactionID).toSerializable(), getRef()); + expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class); + + InOrder inOrder = inOrder(cohort); + inOrder.verify(cohort).canCommit(); + inOrder.verify(cohort).preCommit(); + inOrder.verify(cohort).commit(); + + // Use MBean for verification + // Committed transaction count should increase as usual + assertEquals(1,shard.underlyingActor().getShardMBean().getCommittedTransactionsCount()); + + // Commit index should advance as we do not have an empty modification + assertEquals(0, shard.underlyingActor().getShardMBean().getCommitIndex()); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + + } + }; + } + @Test public void testCommitPhaseFailure() throws Throwable { new ShardTestKit(getSystem()) {{ @@ -954,26 +931,24 @@ public class ShardTest extends AbstractActorTest { waitUntilLeader(shard); final FiniteDuration duration = duration("5 seconds"); - final Timeout timeout = new Timeout(duration); - InMemoryDOMDataStore dataStore = shard.underlyingActor().getDataStore(); final String transactionID = "tx1"; - final CountDownLatch abortComplete = new CountDownLatch(1); Function> preCommit = new Function>() { @Override public ListenableFuture apply(final DOMStoreThreePhaseCommitCohort cohort) { ListenableFuture preCommitFuture = cohort.preCommit(); - Future abortFuture = Patterns.ask(shard, - new AbortTransaction(transactionID).toSerializable(), timeout); - abortFuture.onComplete(new OnComplete() { - @Override - public void onComplete(final Throwable e, final Object resp) { - abortComplete.countDown(); - } - }, getSystem().dispatcher()); + // Simulate an AbortTransaction message occurring during replication, after + // persisting and before finishing the commit to the in-memory store. + // We have no followers so due to optimizations in the RaftActor, it does not + // attempt replication and thus we can't send an AbortTransaction message b/c + // it would be processed too late after CommitTransaction completes. So we'll + // simulate an AbortTransaction message occurring during replication by calling + // the shard directly. + // + shard.underlyingActor().doAbortTransaction(transactionID, null); return preCommitFuture; } @@ -993,14 +968,14 @@ public class ShardTest extends AbstractActorTest { expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); - Future commitFuture = Patterns.ask(shard, - new CommitTransaction(transactionID).toSerializable(), timeout); - - assertEquals("Abort complete", true, abortComplete.await(5, TimeUnit.SECONDS)); - - Await.result(commitFuture, duration); + shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef()); + expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); 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 + // canCommit and preCommit and persisted the data. assertNotNull(TestModel.TEST_QNAME.getLocalName() + " not found", node); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); @@ -1009,7 +984,7 @@ public class ShardTest extends AbstractActorTest { @Test public void testTransactionCommitTimeout() throws Throwable { - dataStoreContext = DatastoreContext.newBuilder().shardTransactionCommitTimeoutInSeconds(1).build(); + dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1); new ShardTestKit(getSystem()) {{ final TestActorRef shard = TestActorRef.create(getSystem(), @@ -1081,7 +1056,7 @@ public class ShardTest extends AbstractActorTest { @Test public void testTransactionCommitQueueCapacityExceeded() throws Throwable { - dataStoreContext = DatastoreContext.newBuilder().shardTransactionCommitQueueCapacity(1).build(); + dataStoreContextBuilder.shardTransactionCommitQueueCapacity(1); new ShardTestKit(getSystem()) {{ final TestActorRef shard = TestActorRef.create(getSystem(), @@ -1214,15 +1189,7 @@ public class ShardTest extends AbstractActorTest { // 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)); + Await.ready(canCommitFuture, duration); InOrder inOrder = inOrder(cohort1, cohort2); inOrder.verify(cohort1).canCommit(); @@ -1233,19 +1200,54 @@ public class ShardTest extends AbstractActorTest { } @Test - public void testCreateSnapshot() throws IOException, InterruptedException { - testCreateSnapshot(true, "testCreateSnapshot"); + public void testCreateSnapshot() throws Exception { + testCreateSnapshot(true, "testCreateSnapshot"); } @Test - public void testCreateSnapshotWithNonPersistentData() throws IOException, InterruptedException { + public void testCreateSnapshotWithNonPersistentData() throws Exception { testCreateSnapshot(false, "testCreateSnapshotWithNonPersistentData"); } @SuppressWarnings("serial") - public void testCreateSnapshot(final boolean persistent, final String shardActorName) throws IOException, InterruptedException { - final DatastoreContext dataStoreContext = DatastoreContext.newBuilder(). - shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(persistent).build(); + 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(); + } + + @Override + public void persist(T o, Procedure procedure) { + delegate.persist(o, procedure); + } + + @Override + public void saveSnapshot(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); + } + } + + dataStoreContextBuilder.persistent(persistent); new ShardTestKit(getSystem()) {{ final AtomicReference latch = new AtomicReference<>(new CountDownLatch(1)); @@ -1253,7 +1255,19 @@ public class ShardTest extends AbstractActorTest { @Override public Shard create() throws Exception { return new Shard(shardID, Collections.emptyMap(), - dataStoreContext, SCHEMA_CONTEXT) { + newDatastoreContext(), SCHEMA_CONTEXT) { + + DelegatingPersistentDataProvider delegating; + + @Override + protected DataPersistenceProvider persistence() { + if(delegating == null) { + delegating = new DelegatingPersistentDataProvider(super.persistence()); + } + + return delegating; + } + @Override protected void commitSnapshot(final long sequenceNumber) { super.commitSnapshot(sequenceNumber); @@ -1268,16 +1282,40 @@ public class ShardTest extends AbstractActorTest { waitUntilLeader(shard); - shard.tell(new CaptureSnapshot(-1,-1,-1,-1), getRef()); + writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + + NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build()); + + CaptureSnapshot capture = new CaptureSnapshot(-1, -1, -1, -1, -1, -1); + shard.tell(capture, getRef()); assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS)); + assertTrue("Invalid saved snapshot " + savedSnapshot.get(), + savedSnapshot.get() instanceof Snapshot); + + verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot); + latch.set(new CountDownLatch(1)); - shard.tell(new CaptureSnapshot(-1,-1,-1,-1), getRef()); + savedSnapshot.set(null); + + shard.tell(capture, getRef()); assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS)); + assertTrue("Invalid saved snapshot " + savedSnapshot.get(), + savedSnapshot.get() instanceof Snapshot); + + verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot); + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + } + + private void verifySnapshot(Snapshot snapshot, NormalizedNode expectedRoot) { + + NormalizedNode actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState()); + assertEquals("Root node", expectedRoot, actual); + }}; } @@ -1342,83 +1380,87 @@ public class ShardTest extends AbstractActorTest { shard2.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; - } + @Test + public void testOnDatastoreContext() { + new ShardTestKit(getSystem()) {{ + dataStoreContextBuilder.persistent(true); - private NormalizedNode readStore(final InMemoryDOMDataStore store) throws ReadFailedException { - DOMStoreReadTransaction transaction = store.newReadOnlyTransaction(); - CheckedFuture>, ReadFailedException> read = - transaction.read(YangInstanceIdentifier.builder().build()); + TestActorRef shard = TestActorRef.create(getSystem(), newShardProps(), "testOnDatastoreContext"); - Optional> optional = read.checkedGet(); + assertEquals("isRecoveryApplicable", true, + shard.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable()); - NormalizedNode normalizedNode = optional.get(); + waitUntilLeader(shard); - transaction.close(); + shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender()); - return normalizedNode; - } + assertEquals("isRecoveryApplicable", false, + shard.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable()); - 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) { - } - } + shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender()); - private AsyncDataChangeListener> noOpDataChangeListener() { - return new AsyncDataChangeListener>() { - @Override - public void onDataChanged( - final AsyncDataChangeEvent> change) { + assertEquals("isRecoveryApplicable", true, + shard.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable()); - } - }; + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; } - static NormalizedNode readStore(final TestActorRef shard, final YangInstanceIdentifier id) - throws ExecutionException, InterruptedException { - DOMStoreReadTransaction transaction = shard.underlyingActor().getDataStore().newReadOnlyTransaction(); + @Test + public void testRegisterRoleChangeListener() throws Exception { + new ShardTestKit(getSystem()) { + { + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testRegisterRoleChangeListener"); + + waitUntilLeader(shard); + + TestActorRef listener = + TestActorRef.create(getSystem(), Props.create(MessageCollectorActor.class)); - CheckedFuture>, ReadFailedException> future = - transaction.read(id); + shard.tell(new RegisterRoleChangeListener(), listener); - Optional> optional = future.get(); - NormalizedNode node = optional.isPresent()? optional.get() : null; + // 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); - transaction.close(); + List allMatching = MessageCollectorActor.getAllMatching(listener, RegisterRoleChangeListenerReply.class); - return node; + assertEquals(1, allMatching.size()); + } + }; } - private void writeToStore(final TestActorRef shard, final YangInstanceIdentifier id, final NormalizedNode node) - throws ExecutionException, InterruptedException { - DOMStoreWriteTransaction transaction = shard.underlyingActor().getDataStore().newWriteOnlyTransaction(); + @Test + public void testFollowerInitialSyncStatus() throws Exception { + final TestActorRef shard = TestActorRef.create(getSystem(), + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testFollowerInitialSyncStatus"); - transaction.write(id, node); + shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational")); - DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready(); - commitCohort.preCommit().get(); - commitCohort.commit().get(); - } + assertEquals(false, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus()); - @SuppressWarnings("serial") - private static final class DelegatingShardCreator implements Creator { - private final Creator delegate; + shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational")); - DelegatingShardCreator(final Creator delegate) { - this.delegate = delegate; - } + assertEquals(true, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus()); + + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + } - @Override - public Shard create() throws Exception { - return delegate.create(); + 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) { } } }