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=d4dcc9cda267943b6b48406757fd359fc820f63b;hp=51063c8d2cc5fe6b2f9e2eb8b4a16d7ed62e30b3;hb=a47dd7a5d21ca68804a6d0e2e3ca765f223c2ef4;hpb=892a6ca966046fd790bdf8a64dccb456a3ece8b4 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 51063c8d2c..d4dcc9cda2 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 @@ -13,11 +13,11 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.doReturn; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.reset; -import static org.mockito.Mockito.verify; import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION; import akka.actor.ActorRef; import akka.actor.ActorSelection; @@ -30,12 +30,13 @@ import akka.pattern.Patterns; import akka.persistence.SaveSnapshotSuccess; import akka.testkit.TestActorRef; import akka.util.Timeout; -import com.google.common.base.Function; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.base.Stopwatch; +import com.google.common.primitives.UnsignedLong; import com.google.common.util.concurrent.Uninterruptibles; +import java.io.IOException; import java.util.Collections; import java.util.HashSet; +import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -45,6 +46,9 @@ 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.access.concepts.LocalHistoryIdentifier; +import org.opendaylight.controller.cluster.access.concepts.MemberName; +import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier; import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats; @@ -58,6 +62,7 @@ import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction; import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction; import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply; +import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction; import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved; import org.opendaylight.controller.cluster.datastore.messages.ReadData; import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply; @@ -69,13 +74,13 @@ import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeCh 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; import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification; import org.opendaylight.controller.cluster.datastore.modification.WriteModification; +import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot; +import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnapshot; 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; import org.opendaylight.controller.cluster.raft.RaftActorContext; @@ -83,14 +88,19 @@ 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.base.messages.TimeoutNow; import org.opendaylight.controller.cluster.raft.client.messages.FindLeader; import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply; +import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState; +import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState; import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.messages.ServerRemoved; import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy; +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.MessageCollectorActor; import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper; @@ -102,11 +112,10 @@ import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode; 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.DataTreeCandidateNode; -import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip; +import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate; 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.api.schema.tree.TipProducingDataTree; import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType; import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes; import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder; @@ -122,7 +131,7 @@ public class ShardTest extends AbstractShardTest { public void testRegisterChangeListener() throws Exception { new ShardTestKit(getSystem()) {{ final TestActorRef shard = actorFactory.createTestActor( - newShardProps(), "testRegisterChangeListener"); + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testRegisterChangeListener"); waitUntilLeader(shard); @@ -174,7 +183,7 @@ public class ShardTest extends AbstractShardTest { // it does do a persist) return new Shard(newShardBuilder()) { @Override - public void onReceiveCommand(final Object message) throws Exception { + public void handleCommand(final Object message) { if(message instanceof ElectionTimeout && firstElectionTimeout) { // Got the first ElectionTimeout. We don't forward it to the // base Shard yet until we've sent the RegisterChangeListener @@ -197,7 +206,7 @@ public class ShardTest extends AbstractShardTest { onFirstElectionTimeout.countDown(); } else { - super.onReceiveCommand(message); + super.handleCommand(message); } } }; @@ -248,7 +257,7 @@ public class ShardTest extends AbstractShardTest { public void testRegisterDataTreeChangeListener() throws Exception { new ShardTestKit(getSystem()) {{ final TestActorRef shard = actorFactory.createTestActor( - newShardProps(), "testRegisterDataTreeChangeListener"); + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testRegisterDataTreeChangeListener"); waitUntilLeader(shard); @@ -286,7 +295,7 @@ public class ShardTest extends AbstractShardTest { public Shard create() throws Exception { return new Shard(newShardBuilder()) { @Override - public void onReceiveCommand(final Object message) throws Exception { + public void handleCommand(final Object message) { if(message instanceof ElectionTimeout && firstElectionTimeout) { firstElectionTimeout = false; final ActorRef self = getSelf(); @@ -301,7 +310,7 @@ public class ShardTest extends AbstractShardTest { onFirstElectionTimeout.countDown(); } else { - super.onReceiveCommand(message); + super.handleCommand(message); } } }; @@ -350,7 +359,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(new UpdateSchemaContext(TestModel.createTestContext()), getRef()); - shard.tell(new CreateTransaction("txn-1", TransactionType.READ_ONLY.ordinal(), null, + shard.tell(new CreateTransaction(nextTransactionId(), TransactionType.READ_ONLY.ordinal(), DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef()); final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"), @@ -358,7 +367,7 @@ public class ShardTest extends AbstractShardTest { final String path = reply.getTransactionPath().toString(); assertTrue("Unexpected transaction path " + path, - path.contains("akka://test/user/testCreateTransaction/shard-txn-1")); + path.startsWith("akka://test/user/testCreateTransaction/shard-member-1:ShardTransactionTest@0:")); }}; } @@ -369,7 +378,7 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - shard.tell(new CreateTransaction("txn-1",TransactionType.READ_ONLY.ordinal(), "foobar", + shard.tell(new CreateTransaction(nextTransactionId(),TransactionType.READ_ONLY.ordinal(), DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef()); final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"), @@ -377,59 +386,32 @@ public class ShardTest extends AbstractShardTest { final String path = reply.getTransactionPath().toString(); assertTrue("Unexpected transaction path " + path, - path.contains("akka://test/user/testCreateTransactionOnChain/shard-txn-1")); + path.startsWith("akka://test/user/testCreateTransactionOnChain/shard-member-1:ShardTransactionTest@0:")); }}; } - @SuppressWarnings("serial") @Test public void testPeerAddressResolved() throws Exception { new ShardTestKit(getSystem()) {{ - final CountDownLatch recoveryComplete = new CountDownLatch(1); - class TestShard extends Shard { - TestShard() { - super(Shard.builder().id(shardID).datastoreContext(newDatastoreContext()). - peerAddresses(Collections.singletonMap(shardID.toString(), null)). - schemaContext(SCHEMA_CONTEXT)); - } - - String getPeerAddress(String id) { - return getRaftActorContext().getPeerAddress(id); - } - - @Override - protected void onRecoveryComplete() { - try { - super.onRecoveryComplete(); - } finally { - recoveryComplete.countDown(); - } - } - } - - final TestActorRef shard = actorFactory.createTestActor( - Props.create(new DelegatingShardCreator(new Creator() { - @Override - public TestShard create() throws Exception { - return new TestShard(); - } - })), "testPeerAddressResolved"); - - assertEquals("Recovery complete", true, - Uninterruptibles.awaitUninterruptibly(recoveryComplete, 5, TimeUnit.SECONDS)); + ShardIdentifier peerID = ShardIdentifier.create("inventory", MemberName.forName("member-2"), "config"); + final TestActorRef shard = actorFactory.createTestActor(newShardBuilder(). + peerAddresses(Collections.singletonMap(peerID.toString(), null)).props(). + withDispatcher(Dispatchers.DefaultDispatcherId()), "testPeerAddressResolved"); final String address = "akka://foobar"; - shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID.toString(), address)); + shard.tell(new PeerAddressResolved(peerID.toString(), address), ActorRef.noSender()); - assertEquals("getPeerAddress", address, - ((TestShard) shard.underlyingActor()).getPeerAddress(shardID.toString())); + shard.tell(GetOnDemandRaftState.INSTANCE, getRef()); + OnDemandRaftState state = expectMsgClass(OnDemandRaftState.class); + assertEquals("getPeerAddress", address, state.getPeerAddresses().get(peerID.toString())); }}; } @Test public void testApplySnapshot() throws Exception { - final TestActorRef shard = actorFactory.createTestActor(newShardProps(), "testApplySnapshot"); + final TestActorRef shard = actorFactory.createTestActor(newShardProps(). + withDispatcher(Dispatchers.DefaultDispatcherId()), "testApplySnapshot"); ShardTestKit.waitUntilLeader(shard); @@ -443,22 +425,33 @@ public class ShardTest extends AbstractShardTest { writeToStore(store, TestModel.TEST_PATH, container); - final YangInstanceIdentifier root = YangInstanceIdentifier.builder().build(); + final YangInstanceIdentifier root = YangInstanceIdentifier.EMPTY; final NormalizedNode expected = readStore(store, root); - final Snapshot snapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(expected), + final Snapshot snapshot = Snapshot.create(new MetadataShardDataTreeSnapshot(expected).serialize(), Collections.emptyList(), 1, 2, 3, 4); - shard.underlyingActor().getRaftActorSnapshotCohort().applySnapshot(snapshot.getState()); + shard.tell(new ApplySnapshot(snapshot), ActorRef.noSender()); - final NormalizedNode actual = readStore(shard, root); + Stopwatch sw = Stopwatch.createStarted(); + while(sw.elapsed(TimeUnit.SECONDS) <= 5) { + Uninterruptibles.sleepUninterruptibly(75, TimeUnit.MILLISECONDS); + + try { + assertEquals("Root node", expected, readStore(shard, root)); + return; + } catch(AssertionError e) { + // try again + } + } - assertEquals("Root node", expected, actual); + fail("Snapshot was not applied"); } @Test public void testApplyState() throws Exception { - final TestActorRef shard = actorFactory.createTestActor(newShardProps(), "testApplyState"); + final TestActorRef shard = actorFactory.createTestActor( + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testApplyState"); ShardTestKit.waitUntilLeader(shard); @@ -468,13 +461,24 @@ public class ShardTest extends AbstractShardTest { writeMod.write(TestModel.TEST_PATH, node); writeMod.ready(); - final ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2, - payloadForModification(source, writeMod))); + final TransactionIdentifier tx = nextTransactionId(); + final ApplyState applyState = new ApplyState(null, tx, + new ReplicatedLogImplEntry(1, 2, payloadForModification(source, writeMod, tx))); + + shard.tell(applyState, shard); + + Stopwatch sw = Stopwatch.createStarted(); + while(sw.elapsed(TimeUnit.SECONDS) <= 5) { + Uninterruptibles.sleepUninterruptibly(75, TimeUnit.MILLISECONDS); - shard.underlyingActor().onReceiveCommand(applyState); + final NormalizedNode actual = readStore(shard, TestModel.TEST_PATH); + if(actual != null) { + assertEquals("Applied state", node, actual); + return; + } + } - final NormalizedNode actual = readStore(shard, TestModel.TEST_PATH); - assertEquals("Applied state", node, actual); + fail("State was not applied"); } @Test @@ -488,7 +492,8 @@ public class ShardTest extends AbstractShardTest { InMemoryJournal.addEntry(shardID.toString(), 0, DUMMY_DATA); // Set up the InMemoryJournal. - InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, payloadForModification(source, writeMod))); + InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, + payloadForModification(source, writeMod, nextTransactionId()))); final int nListEntries = 16; final Set listEntryKeys = new HashSet<>(); @@ -503,8 +508,9 @@ public class ShardTest extends AbstractShardTest { 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))); + payloadForModification(source, mod, nextTransactionId()))); } InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2, @@ -522,35 +528,22 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - // Setup 3 simulated transactions with mock cohorts backed by real cohorts. - - final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); + final TransactionIdentifier transactionID1 = nextTransactionId(); + final TransactionIdentifier transactionID2 = nextTransactionId(); + final TransactionIdentifier transactionID3 = nextTransactionId(); - final String transactionID1 = "tx1"; - final MutableCompositeModification modification1 = new MutableCompositeModification(); - final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, - TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1); - - 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); + Map cohortMap = setupCohortDecorator( + shard.underlyingActor(), transactionID1, transactionID2, transactionID3); + final CapturingShardDataTreeCohort cohort1 = cohortMap.get(transactionID1); + final CapturingShardDataTreeCohort cohort2 = cohortMap.get(transactionID2); + final CapturingShardDataTreeCohort cohort3 = cohortMap.get(transactionID3); final long timeoutSec = 5; final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS); final Timeout timeout = new Timeout(duration); - shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); + shard.tell(prepareBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef()); final ReadyTransactionReply readyReply = ReadyTransactionReply.fromSerializable( expectMsgClass(duration, ReadyTransactionReply.class)); assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath()); @@ -562,10 +555,15 @@ public class ShardTest extends AbstractShardTest { expectMsgClass(duration, CanCommitTransactionReply.class)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); - shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); + // Ready 2 more Tx's. + + shard.tell(prepareBatchedModifications(transactionID2, TestModel.OUTER_LIST_PATH, + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef()); + shard.tell(prepareBatchedModifications(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), false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and @@ -626,9 +624,9 @@ public class ShardTest extends AbstractShardTest { } class OnCanCommitFutureComplete extends OnFutureComplete { - private final String transactionID; + private final TransactionIdentifier transactionID; - OnCanCommitFutureComplete(final String transactionID) { + OnCanCommitFutureComplete(final TransactionIdentifier transactionID) { super(CanCommitTransactionReply.class); this.transactionID = transactionID; } @@ -659,16 +657,18 @@ public class ShardTest extends AbstractShardTest { assertEquals("Commits complete", true, done); - 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(); + final InOrder inOrder = inOrder(cohort1.getCanCommit(), cohort1.getPreCommit(), cohort1.getCommit(), + cohort2.getCanCommit(), cohort2.getPreCommit(), cohort2.getCommit(), cohort3.getCanCommit(), + cohort3.getPreCommit(), cohort3.getCommit()); + inOrder.verify(cohort1.getCanCommit()).onSuccess(any(Void.class)); + inOrder.verify(cohort1.getPreCommit()).onSuccess(any(DataTreeCandidate.class)); + inOrder.verify(cohort1.getCommit()).onSuccess(any(UnsignedLong.class)); + inOrder.verify(cohort2.getCanCommit()).onSuccess(any(Void.class)); + inOrder.verify(cohort2.getPreCommit()).onSuccess(any(DataTreeCandidate.class)); + inOrder.verify(cohort2.getCommit()).onSuccess(any(UnsignedLong.class)); + inOrder.verify(cohort3.getCanCommit()).onSuccess(any(Void.class)); + inOrder.verify(cohort3.getPreCommit()).onSuccess(any(DataTreeCandidate.class)); + inOrder.verify(cohort3.getCommit()).onSuccess(any(UnsignedLong.class)); // Verify data in the data store. @@ -687,23 +687,9 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionID = "tx"; + final TransactionIdentifier transactionID = nextTransactionId(); 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, @@ -728,16 +714,11 @@ public class ShardTest extends AbstractShardTest { expectMsgClass(duration, CanCommitTransactionReply.class)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); - // Send the CanCommitTransaction message. + // Send the CommitTransaction message. shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); expectMsgClass(duration, CommitTransactionReply.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. verifyOuterListEntry(shard, 1); @@ -753,23 +734,9 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionID = "tx"; + final TransactionIdentifier transactionID = nextTransactionId(); 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, @@ -788,11 +755,6 @@ public class ShardTest extends AbstractShardTest { expectMsgClass(duration, CommitTransactionReply.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. verifyOuterListEntry(shard, 1); @@ -808,8 +770,8 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionID = "tx1"; - final BatchedModifications batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION, null); + final TransactionIdentifier transactionID = nextTransactionId(); + final BatchedModifications batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION); batched.setReady(true); batched.setTotalMessagesSent(2); @@ -835,20 +797,20 @@ public class ShardTest extends AbstractShardTest { // 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"; + TransactionIdentifier transactionID = nextTransactionId(); 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); + BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION); 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 = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION); batched.setReady(true); batched.setTotalMessagesSent(2); @@ -868,9 +830,9 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionChainID = "txChain"; - final String transactionID1 = "tx1"; - final String transactionID2 = "tx2"; + final LocalHistoryIdentifier historyId = nextHistoryId(); + final TransactionIdentifier transactionID1 = new TransactionIdentifier(historyId, 0); + final TransactionIdentifier transactionID2 = new TransactionIdentifier(historyId, 1); final FiniteDuration duration = duration("5 seconds"); @@ -878,14 +840,13 @@ public class ShardTest extends AbstractShardTest { final ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); final YangInstanceIdentifier path = TestModel.TEST_PATH; - shard.tell(newBatchedModifications(transactionID1, transactionChainID, path, - containerNode, true, false, 1), getRef()); + shard.tell(newBatchedModifications(transactionID1, path, containerNode, true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Create a read Tx on the same chain. shard.tell(new CreateTransaction(transactionID2, TransactionType.READ_ONLY.ordinal(), - transactionChainID, DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef()); + DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef()); final CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class); @@ -936,13 +897,14 @@ public class ShardTest extends AbstractShardTest { }; final TestActorRef shard = actorFactory.createTestActor( - Props.create(new DelegatingShardCreator(creator)), "testOnBatchedModificationsWhenNotLeader"); + Props.create(new DelegatingShardCreator(creator)). + withDispatcher(Dispatchers.DefaultDispatcherId()), "testOnBatchedModificationsWhenNotLeader"); waitUntilLeader(shard); overrideLeaderCalls.set(true); - final BatchedModifications batched = new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, ""); + final BatchedModifications batched = new BatchedModifications(nextTransactionId(), DataStoreVersions.CURRENT_VERSION); shard.tell(batched, ActorRef.noSender()); @@ -961,24 +923,29 @@ public class ShardTest extends AbstractShardTest { waitUntilNoLeader(shard); - shard.tell(new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, ""), getRef()); + final TransactionIdentifier txId = nextTransactionId(); + shard.tell(new BatchedModifications(txId, DataStoreVersions.CURRENT_VERSION), getRef()); Failure failure = expectMsgClass(Failure.class); assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass()); - shard.tell(prepareForwardedReadyTransaction(mock(ShardDataTreeCohort.class), "tx", - DataStoreVersions.CURRENT_VERSION, true), getRef()); + shard.tell(prepareForwardedReadyTransaction(shard, txId, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); failure = expectMsgClass(Failure.class); assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass()); - shard.tell(new ReadyLocalTransaction("tx", mock(DataTreeModification.class), true), getRef()); + shard.tell(new ReadyLocalTransaction(txId, mock(DataTreeModification.class), true), getRef()); failure = expectMsgClass(Failure.class); assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass()); }}; } @Test - public void testReadyWithImmediateCommit() throws Exception{ + public void testReadyWithReadWriteImmediateCommit() throws Exception{ testReadyWithImmediateCommit(true); + } + + @Test + public void testReadyWithWriteOnlyImmediateCommit() throws Exception{ testReadyWithImmediateCommit(false); } @@ -990,24 +957,16 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - - final String transactionID = "tx1"; - final MutableCompositeModification modification = new MutableCompositeModification(); + final TransactionIdentifier transactionID = nextTransactionId(); 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"); - - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification, true), getRef()); - - expectMsgClass(duration, CommitTransactionReply.class); + if(readWrite) { + shard.tell(prepareForwardedReadyTransaction(shard, transactionID, TestModel.TEST_PATH, + containerNode, true), getRef()); + } else { + shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH, containerNode, true), getRef()); + } - final InOrder inOrder = inOrder(cohort); - inOrder.verify(cohort).canCommit(); - inOrder.verify(cohort).preCommit(); - inOrder.verify(cohort).commit(); + expectMsgClass(duration("5 seconds"), CommitTransactionReply.class); final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode); @@ -1032,7 +991,7 @@ public class ShardTest extends AbstractShardTest { final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(); new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification); - final String txId = "tx1"; + final TransactionIdentifier txId = nextTransactionId(); modification.ready(); final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, true); @@ -1063,8 +1022,8 @@ public class ShardTest extends AbstractShardTest { 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 TransactionIdentifier txId = nextTransactionId(); + modification.ready(); final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, false); shard.tell(readyMessage, getRef()); @@ -1089,33 +1048,22 @@ public class ShardTest extends AbstractShardTest { } @Test - public void testCommitWithPersistenceDisabled() throws Throwable { - testCommitWithPersistenceDisabled(true); - testCommitWithPersistenceDisabled(false); - } - - private void testCommitWithPersistenceDisabled(final boolean readWrite) throws Throwable { + public void testReadWriteCommitWithPersistenceDisabled() throws Throwable { dataStoreContextBuilder.persistent(false); new ShardTestKit(getSystem()) {{ final TestActorRef shard = actorFactory.createTestActor( newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testCommitWithPersistenceDisabled-" + readWrite); + "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"); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef()); + final TransactionIdentifier transactionID = nextTransactionId(); + final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH, containerNode, false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message. @@ -1130,168 +1078,152 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); expectMsgClass(duration, 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); }}; } @Test - public void testCommitWhenTransactionHasNoModifications() { + public void testReadWriteCommitWhenTransactionHasNoModifications() { testCommitWhenTransactionHasNoModifications(true); + } + + @Test + public void testWriteOnlyCommitWhenTransactionHasNoModifications() { testCommitWhenTransactionHasNoModifications(false); } private void testCommitWhenTransactionHasNoModifications(final boolean readWrite){ // 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 = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testCommitWhenTransactionHasNoModifications-" + readWrite); + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = actorFactory.createTestActor( + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCommitWhenTransactionHasNoModifications-" + readWrite); - waitUntilLeader(shard); + waitUntilLeader(shard); - 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(); + final TransactionIdentifier transactionID = nextTransactionId(); - final FiniteDuration duration = duration("5 seconds"); + final FiniteDuration duration = duration("5 seconds"); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef()); - expectMsgClass(duration, ReadyTransactionReply.class); + if(readWrite) { + ReadWriteShardDataTreeTransaction rwTx = shard.underlyingActor().getDataStore(). + newReadWriteTransaction(transactionID); + shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, rwTx, false), getRef()); + } else { + shard.tell(prepareBatchedModifications(transactionID, new MutableCompositeModification()), getRef()); + } - // Send the CanCommitTransaction message. + expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); - final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( - expectMsgClass(duration, CanCommitTransactionReply.class)); - assertEquals("Can commit", true, canCommitReply.getCanCommit()); + // Send the CanCommitTransaction message. - shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); - expectMsgClass(duration, CommitTransactionReply.class); + shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + expectMsgClass(duration, CanCommitTransactionReply.class)); + assertEquals("Can commit", true, canCommitReply.getCanCommit()); - final InOrder inOrder = inOrder(cohort); - inOrder.verify(cohort).canCommit(); - inOrder.verify(cohort).preCommit(); - inOrder.verify(cohort).commit(); + shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, CommitTransactionReply.class); - shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef()); - final ShardStats shardStats = expectMsgClass(duration, ShardStats.class); + 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,shardStats.getCommittedTransactionsCount()); + // Use MBean for verification + // Committed transaction count should increase as usual + assertEquals(1,shardStats.getCommittedTransactionsCount()); - // Commit index should not advance because this does not go into the journal - assertEquals(-1, shardStats.getCommitIndex()); - } - }; + // Commit index should not advance because this does not go into the journal + assertEquals(-1, shardStats.getCommitIndex()); + }}; } @Test - public void testCommitWhenTransactionHasModifications() { + public void testReadWriteCommitWhenTransactionHasModifications() throws Exception { testCommitWhenTransactionHasModifications(true); - testCommitWhenTransactionHasModifications(false); } - private void testCommitWhenTransactionHasModifications(final boolean readWrite){ - new ShardTestKit(getSystem()) { - { - final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testCommitWhenTransactionHasModifications-" + readWrite); + @Test + public void testWriteOnlyCommitWhenTransactionHasModifications() throws Exception { + testCommitWhenTransactionHasModifications(false); + } - waitUntilLeader(shard); + private void testCommitWhenTransactionHasModifications(final boolean readWrite) throws Exception { + new ShardTestKit(getSystem()) {{ + final TipProducingDataTree dataTree = createDelegatingMockDataTree(); + final TestActorRef shard = actorFactory.createTestActor( + newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCommitWhenTransactionHasModifications-" + readWrite); - final String transactionID = "tx1"; - final MutableCompositeModification modification = new MutableCompositeModification(); - modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build())); - 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(); + waitUntilLeader(shard); - final FiniteDuration duration = duration("5 seconds"); + final FiniteDuration duration = duration("5 seconds"); + final TransactionIdentifier transactionID = nextTransactionId(); + + if(readWrite) { + shard.tell(prepareForwardedReadyTransaction(shard, transactionID, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef()); + } else { + shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef()); + } - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef()); - expectMsgClass(duration, ReadyTransactionReply.class); + expectMsgClass(duration, ReadyTransactionReply.class); - // Send the CanCommitTransaction message. + // Send the CanCommitTransaction message. - shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); - final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( - expectMsgClass(duration, CanCommitTransactionReply.class)); - assertEquals("Can commit", true, canCommitReply.getCanCommit()); + shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); + final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + expectMsgClass(duration, CanCommitTransactionReply.class)); + assertEquals("Can commit", true, canCommitReply.getCanCommit()); - shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); - expectMsgClass(duration, CommitTransactionReply.class); + shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, CommitTransactionReply.class); - final InOrder inOrder = inOrder(cohort); - inOrder.verify(cohort).canCommit(); - inOrder.verify(cohort).preCommit(); - inOrder.verify(cohort).commit(); + final InOrder inOrder = inOrder(dataTree); + inOrder.verify(dataTree).validate(any(DataTreeModification.class)); + inOrder.verify(dataTree).prepare(any(DataTreeModification.class)); + inOrder.verify(dataTree).commit(any(DataTreeCandidate.class)); - shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef()); - final ShardStats shardStats = expectMsgClass(duration, ShardStats.class); + 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, shardStats.getCommittedTransactionsCount()); + // Use MBean for verification + // Committed transaction count should increase as usual + assertEquals(1, shardStats.getCommittedTransactionsCount()); - // Commit index should advance as we do not have an empty modification - assertEquals(0, shardStats.getCommitIndex()); - } - }; + // Commit index should advance as we do not have an empty modification + assertEquals(0, shardStats.getCommitIndex()); + }}; } @Test public void testCommitPhaseFailure() throws Throwable { - testCommitPhaseFailure(true); - testCommitPhaseFailure(false); - } - - private void testCommitPhaseFailure(final boolean readWrite) throws Throwable { new ShardTestKit(getSystem()) {{ + final TipProducingDataTree dataTree = createDelegatingMockDataTree(); final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testCommitPhaseFailure-" + readWrite); + newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCommitPhaseFailure"); waitUntilLeader(shard); + final FiniteDuration duration = duration("5 seconds"); + final Timeout timeout = new Timeout(duration); + // Setup 2 simulated transactions with mock cohorts. The first one fails in the // commit phase. - 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.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 MutableCompositeModification modification2 = new MutableCompositeModification(); - final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); - - final FiniteDuration duration = duration("5 seconds"); - final Timeout timeout = new Timeout(duration); + doThrow(new RuntimeException("mock commit failure")).when(dataTree).commit(any(DataTreeCandidate.class)); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); + final TransactionIdentifier transactionID1 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message for the first Tx. @@ -1325,46 +1257,37 @@ public class ShardTest extends AbstractShardTest { 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(cohort1).commit(); - inOrder.verify(cohort2).canCommit(); + final InOrder inOrder = inOrder(dataTree); + inOrder.verify(dataTree).validate(any(DataTreeModification.class)); + inOrder.verify(dataTree).prepare(any(DataTreeModification.class)); + inOrder.verify(dataTree).commit(any(DataTreeCandidate.class)); + inOrder.verify(dataTree).validate(any(DataTreeModification.class)); }}; } @Test public void testPreCommitPhaseFailure() throws Throwable { - testPreCommitPhaseFailure(true); - testPreCommitPhaseFailure(false); - } - - private void testPreCommitPhaseFailure(final boolean readWrite) throws Throwable { new ShardTestKit(getSystem()) {{ + final TipProducingDataTree dataTree = createDelegatingMockDataTree(); final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testPreCommitPhaseFailure-" + readWrite); + newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testPreCommitPhaseFailure"); waitUntilLeader(shard); - 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(); - - final String transactionID2 = "tx2"; - final MutableCompositeModification modification2 = new MutableCompositeModification(); - final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); - final FiniteDuration duration = duration("5 seconds"); final Timeout timeout = new Timeout(duration); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); + doThrow(new RuntimeException("mock preCommit failure")).when(dataTree).prepare(any(DataTreeModification.class)); + + final TransactionIdentifier transactionID1 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message for the first Tx. @@ -1398,35 +1321,31 @@ public class ShardTest extends AbstractShardTest { 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(); + final InOrder inOrder = inOrder(dataTree); + inOrder.verify(dataTree).validate(any(DataTreeModification.class)); + inOrder.verify(dataTree).prepare(any(DataTreeModification.class)); + inOrder.verify(dataTree).validate(any(DataTreeModification.class)); }}; } @Test public void testCanCommitPhaseFailure() throws Throwable { - testCanCommitPhaseFailure(true); - testCanCommitPhaseFailure(false); - } - - private void testCanCommitPhaseFailure(final boolean readWrite) throws Throwable { new ShardTestKit(getSystem()) {{ + final TipProducingDataTree dataTree = createDelegatingMockDataTree(); final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testCanCommitPhaseFailure-" + readWrite); + newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testCanCommitPhaseFailure"); waitUntilLeader(shard); final FiniteDuration duration = duration("5 seconds"); + final TransactionIdentifier transactionID1 = nextTransactionId(); - 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(); + doThrow(new DataValidationFailedException(YangInstanceIdentifier.EMPTY, "mock canCommit failure")). + doNothing().when(dataTree).validate(any(DataTreeModification.class)); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification), getRef()); + shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message. @@ -1436,12 +1355,9 @@ public class ShardTest extends AbstractShardTest { // 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(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification), getRef()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef()); @@ -1451,54 +1367,6 @@ public class ShardTest extends AbstractShardTest { }}; } - @Test - public void testCanCommitPhaseFalseResponse() throws Throwable { - testCanCommitPhaseFalseResponse(true); - testCanCommitPhaseFalseResponse(false); - } - - private void testCanCommitPhaseFalseResponse(final boolean readWrite) throws Throwable { - new ShardTestKit(getSystem()) {{ - final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testCanCommitPhaseFalseResponse-" + readWrite); - - 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(); - - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification), getRef()); - expectMsgClass(duration, ReadyTransactionReply.class); - - // Send the CanCommitTransaction message. - - shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); - CanCommitTransactionReply reply = CanCommitTransactionReply.fromSerializable( - expectMsgClass(CanCommitTransactionReply.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(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification), getRef()); - expectMsgClass(duration, ReadyTransactionReply.class); - - shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef()); - reply = CanCommitTransactionReply.fromSerializable( - expectMsgClass(CanCommitTransactionReply.class)); - assertEquals("getCanCommit", true, reply.getCanCommit()); - }}; - } - @Test public void testImmediateCommitWithCanCommitPhaseFailure() throws Throwable { testImmediateCommitWithCanCommitPhaseFailure(true); @@ -1507,138 +1375,81 @@ public class ShardTest extends AbstractShardTest { private void testImmediateCommitWithCanCommitPhaseFailure(final boolean readWrite) throws Throwable { new ShardTestKit(getSystem()) {{ + final TipProducingDataTree dataTree = createDelegatingMockDataTree(); final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), + newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()), "testImmediateCommitWithCanCommitPhaseFailure-" + readWrite); waitUntilLeader(shard); + doThrow(new DataValidationFailedException(YangInstanceIdentifier.EMPTY, "mock canCommit failure")). + doNothing().when(dataTree).validate(any(DataTreeModification.class)); + 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(); + final TransactionIdentifier transactionID1 = nextTransactionId(); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification, true), getRef()); + if(readWrite) { + shard.tell(prepareForwardedReadyTransaction(shard, transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); + } else { + shard.tell(prepareBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), 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(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + if(readWrite) { + shard.tell(prepareForwardedReadyTransaction(shard, transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); + } else { + shard.tell(prepareBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef()); + } expectMsgClass(duration, CommitTransactionReply.class); }}; } + @SuppressWarnings("serial") @Test - public void testImmediateCommitWithCanCommitPhaseFalseResponse() throws Throwable { - testImmediateCommitWithCanCommitPhaseFalseResponse(true); - testImmediateCommitWithCanCommitPhaseFalseResponse(false); - } - - private void testImmediateCommitWithCanCommitPhaseFalseResponse(final boolean readWrite) throws Throwable { + public void testAbortWithCommitPending() throws Throwable { new ShardTestKit(getSystem()) {{ - final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testImmediateCommitWithCanCommitPhaseFalseResponse-" + readWrite); - - 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(); - - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification, 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(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef()); - - expectMsgClass(duration, CommitTransactionReply.class); - }}; - } + final Creator creator = new Creator() { + @Override + public Shard create() throws Exception { + return new Shard(newShardBuilder()) { + @Override + void persistPayload(final TransactionIdentifier transactionId, final Payload payload) { + // Simulate an AbortTransaction message occurring during replication, after + // persisting and before finishing the commit to the in-memory store. - @Test - public void testAbortBeforeFinishCommit() throws Throwable { - testAbortBeforeFinishCommit(true); - testAbortBeforeFinishCommit(false); - } + doAbortTransaction(transactionId, null); + super.persistPayload(transactionId, payload); + } + }; + } + }; - private void testAbortBeforeFinishCommit(final boolean readWrite) throws Throwable { - new ShardTestKit(getSystem()) {{ final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testAbortBeforeFinishCommit-" + readWrite); + Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), + "testAbortWithCommitPending"); waitUntilLeader(shard); final FiniteDuration duration = duration("5 seconds"); - final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - - final String transactionID = "tx1"; - final Function> preCommit = - new Function>() { - @Override - 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. - // 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; - } - }; - final MutableCompositeModification modification = new MutableCompositeModification(); - final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort1", dataStore, - TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), - modification, preCommit); + final TransactionIdentifier transactionID = nextTransactionId(); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef()); + shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); - final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( - expectMsgClass(duration, CanCommitTransactionReply.class)); - assertEquals("Can commit", true, canCommitReply.getCanCommit()); + expectMsgClass(duration, CanCommitTransactionReply.class); shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); expectMsgClass(duration, CommitTransactionReply.class); @@ -1654,55 +1465,33 @@ public class ShardTest extends AbstractShardTest { @Test public void testTransactionCommitTimeout() throws Throwable { - testTransactionCommitTimeout(true); - testTransactionCommitTimeout(false); - } - - private void testTransactionCommitTimeout(final boolean readWrite) throws Throwable { dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1); - new ShardTestKit(getSystem()) {{ final TestActorRef shard = actorFactory.createTestActor( newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testTransactionCommitTimeout-" + readWrite); + "testTransactionCommitTimeout"); waitUntilLeader(shard); 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 1st Tx - will timeout - - 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); + // Ready 2 Tx's - the first will timeout - // 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(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); + final TransactionIdentifier transactionID1 = nextTransactionId(); + shard.tell(prepareBatchedModifications(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), false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + final YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) + .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(); + shard.tell(prepareBatchedModifications(transactionID2, listNodePath, + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2), false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // canCommit 1st Tx. We don't send the commit so it should timeout. @@ -1730,71 +1519,73 @@ public class ShardTest extends AbstractShardTest { }}; } - @Test - public void testTransactionCommitQueueCapacityExceeded() throws Throwable { - dataStoreContextBuilder.shardTransactionCommitQueueCapacity(2); - - new ShardTestKit(getSystem()) {{ - final TestActorRef shard = actorFactory.createTestActor( - newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testTransactionCommitQueueCapacityExceeded"); - - 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); - - 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, - TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification3); - - // Ready the Tx's - - shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); - expectMsgClass(duration, ReadyTransactionReply.class); - - shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); - expectMsgClass(duration, ReadyTransactionReply.class); - - // The 3rd Tx should exceed queue capacity and fail. - - shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef()); - expectMsgClass(duration, akka.actor.Status.Failure.class); - - // canCommit 1st Tx. - - shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); - expectMsgClass(duration, CanCommitTransactionReply.class); - - // canCommit the 2nd Tx - it should get queued. - - shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef()); - - // canCommit the 3rd Tx - should exceed queue capacity and fail. - - shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef()); - expectMsgClass(duration, akka.actor.Status.Failure.class); - }}; - } +// @Test +// @Ignore +// public void testTransactionCommitQueueCapacityExceeded() throws Throwable { +// dataStoreContextBuilder.shardTransactionCommitQueueCapacity(2); +// +// new ShardTestKit(getSystem()) {{ +// final TestActorRef shard = actorFactory.createTestActor( +// newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), +// "testTransactionCommitQueueCapacityExceeded"); +// +// waitUntilLeader(shard); +// +// final FiniteDuration duration = duration("5 seconds"); +// +// final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); +// +// final TransactionIdentifier transactionID1 = nextTransactionId(); +// final MutableCompositeModification modification1 = new MutableCompositeModification(); +// final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, +// TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), transactionID1, +// modification1); +// +// final TransactionIdentifier transactionID2 = nextTransactionId(); +// final MutableCompositeModification modification2 = new MutableCompositeModification(); +// final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore, +// TestModel.OUTER_LIST_PATH, +// ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), transactionID2, +// modification2); +// +// final TransactionIdentifier transactionID3 = nextTransactionId(); +// final MutableCompositeModification modification3 = new MutableCompositeModification(); +// final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore, +// TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), transactionID3, +// modification3); +// +// // Ready the Tx's +// +// shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); +// expectMsgClass(duration, ReadyTransactionReply.class); +// +// shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); +// expectMsgClass(duration, ReadyTransactionReply.class); +// +// // The 3rd Tx should exceed queue capacity and fail. +// +// shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef()); +// expectMsgClass(duration, akka.actor.Status.Failure.class); +// +// // canCommit 1st Tx. +// +// shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); +// expectMsgClass(duration, CanCommitTransactionReply.class); +// +// // canCommit the 2nd Tx - it should get queued. +// +// shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef()); +// +// // canCommit the 3rd Tx - should exceed queue capacity and fail. +// +// shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef()); +// expectMsgClass(duration, akka.actor.Status.Failure.class); +// }}; +// } @Test public void testTransactionCommitWithPriorExpiredCohortEntries() throws Throwable { - dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1); - + dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1); new ShardTestKit(getSystem()) {{ final TestActorRef shard = actorFactory.createTestActor( newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), @@ -1804,30 +1595,19 @@ public class ShardTest extends AbstractShardTest { 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(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); + final TransactionIdentifier transactionID1 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), 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(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), 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(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef()); + final TransactionIdentifier transactionID3 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID3, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // All Tx's are readied. We'll send canCommit for the last one but not the others. The others @@ -1840,8 +1620,7 @@ public class ShardTest extends AbstractShardTest { @Test public void testTransactionCommitWithSubsequentExpiredCohortEntry() throws Throwable { - dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1); - + dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1); new ShardTestKit(getSystem()) {{ final TestActorRef shard = actorFactory.createTestActor( newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), @@ -1853,38 +1632,31 @@ public class ShardTest extends AbstractShardTest { 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(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); + final TransactionIdentifier transactionID1 = nextTransactionId(); + shard.tell(prepareBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - // CanCommit the first one so it's the current in-progress CohortEntry. + // CanCommit the first Tx so it's the current in-progress Tx. shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); expectMsgClass(duration, CanCommitTransactionReply.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(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + shard.tell(prepareBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Ready the third Tx. - final String transactionID3 = "tx3"; + final TransactionIdentifier transactionID3 = nextTransactionId(); final DataTreeModification modification3 = dataStore.newModification(); new WriteModification(TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME)) .apply(modification3); - modification3.ready(); + 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 @@ -1909,51 +1681,39 @@ public class ShardTest extends AbstractShardTest { newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testCanCommitBeforeReadyFailure"); - shard.tell(new CanCommitTransaction("tx", CURRENT_VERSION).toSerializable(), getRef()); + shard.tell(new CanCommitTransaction(nextTransactionId(), CURRENT_VERSION).toSerializable(), getRef()); expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class); }}; } @Test - public void testAbortCurrentTransaction() throws Throwable { - testAbortCurrentTransaction(true); - testAbortCurrentTransaction(false); - } - - private void testAbortCurrentTransaction(final boolean readWrite) throws Throwable { + public void testAbortAfterCanCommit() throws Throwable { new ShardTestKit(getSystem()) {{ final TestActorRef shard = actorFactory.createTestActor( newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), - "testAbortCurrentTransaction-" + readWrite); + "testAbortAfterCanCommit"); waitUntilLeader(shard); - // Setup 2 simulated transactions with mock cohorts. The first one will be aborted. - - 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.immediateFuture(null)).when(cohort1).abort(); - - final String transactionID2 = "tx2"; - final MutableCompositeModification modification2 = new MutableCompositeModification(); - final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); - final FiniteDuration duration = duration("5 seconds"); final Timeout timeout = new Timeout(duration); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); + // Ready 2 transactions - the first one will be aborted. + + final TransactionIdentifier transactionID1 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message for the first Tx. shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); - final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( + CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( expectMsgClass(duration, CanCommitTransactionReply.class)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); @@ -1971,84 +1731,101 @@ public class ShardTest extends AbstractShardTest { // Wait for the 2nd Tx to complete the canCommit phase. - Await.ready(canCommitFuture, duration); - - final InOrder inOrder = inOrder(cohort1, cohort2); - inOrder.verify(cohort1).canCommit(); - inOrder.verify(cohort2).canCommit(); + canCommitReply = (CanCommitTransactionReply) Await.result(canCommitFuture, duration); + assertEquals("Can commit", true, canCommitReply.getCanCommit()); }}; } @Test - public void testAbortQueuedTransaction() throws Throwable { - testAbortQueuedTransaction(true); - testAbortQueuedTransaction(false); - } - - private void testAbortQueuedTransaction(final boolean readWrite) throws Throwable { + public void testAbortAfterReady() throws Throwable { dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1); new ShardTestKit(getSystem()) {{ - final AtomicReference cleaupCheckLatch = new AtomicReference<>(); - @SuppressWarnings("serial") - final Creator creator = new Creator() { - @Override - public Shard create() throws Exception { - return new Shard(newShardBuilder()) { - @Override - public void onReceiveCommand(final Object message) throws Exception { - super.onReceiveCommand(message); - if(message.equals(TX_COMMIT_TIMEOUT_CHECK_MESSAGE)) { - if(cleaupCheckLatch.get() != null) { - cleaupCheckLatch.get().countDown(); - } - } - } - }; - } - }; - final TestActorRef shard = actorFactory.createTestActor( - Props.create(new DelegatingShardCreator(creator)).withDispatcher( - Dispatchers.DefaultDispatcherId()), "testAbortQueuedTransaction-" + readWrite); + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testAbortAfterReady"); waitUntilLeader(shard); - final String transactionID = "tx1"; + final FiniteDuration duration = duration("5 seconds"); + + // Ready a tx. + + final TransactionIdentifier transactionID1 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + // Send the AbortTransaction message. + + shard.tell(new AbortTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, AbortTransactionReply.class); - final MutableCompositeModification modification = new MutableCompositeModification(); - final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort"); - doReturn(Futures.immediateFuture(null)).when(cohort).abort(); + assertEquals("getPendingTxCommitQueueSize", 0, shard.underlyingActor().getPendingTxCommitQueueSize()); + + // Now send CanCommitTransaction - should fail. + + shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); + Throwable failure = expectMsgClass(duration, akka.actor.Status.Failure.class).cause(); + assertTrue("Failure type", failure instanceof IllegalStateException); + + // Ready and CanCommit another and verify success. + + final TransactionIdentifier transactionID2 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); + + shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, CanCommitTransactionReply.class); + }}; + } + + @Test + public void testAbortQueuedTransaction() throws Throwable { + new ShardTestKit(getSystem()) {{ + final TestActorRef shard = actorFactory.createTestActor( + newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testAbortAfterReady"); + + waitUntilLeader(shard); final FiniteDuration duration = duration("5 seconds"); - // Ready the tx. + // Ready 3 tx's. - shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef()); + final TransactionIdentifier transactionID1 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - assertEquals("getPendingTxCommitQueueSize", 1, shard.underlyingActor().getPendingTxCommitQueueSize()); + final TransactionIdentifier transactionID2 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH, + ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); - // Send the AbortTransaction message. + final TransactionIdentifier transactionID3 = nextTransactionId(); + shard.tell(newBatchedModifications(transactionID3, TestModel.OUTER_LIST_PATH, + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), true, false, 1), getRef()); + expectMsgClass(duration, ReadyTransactionReply.class); - shard.tell(new AbortTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); - expectMsgClass(duration, AbortTransactionReply.class); + // Abort the second tx while it's queued. - verify(cohort).abort(); + shard.tell(new AbortTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, AbortTransactionReply.class); - // Verify the tx cohort is removed from queue at the cleanup check interval. + // Commit the other 2. - cleaupCheckLatch.set(new CountDownLatch(1)); - assertEquals("TX_COMMIT_TIMEOUT_CHECK_MESSAGE received", true, - cleaupCheckLatch.get().await(5, TimeUnit.SECONDS)); + shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, CanCommitTransactionReply.class); - assertEquals("getPendingTxCommitQueueSize", 0, shard.underlyingActor().getPendingTxCommitQueueSize()); + shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, CommitTransactionReply.class); - // Now send CanCommitTransaction - should fail. + shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, CanCommitTransactionReply.class); - shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef()); + shard.tell(new CommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef()); + expectMsgClass(duration, CommitTransactionReply.class); - Throwable failure = expectMsgClass(duration, akka.actor.Status.Failure.class).cause(); - assertTrue("Failure type", failure instanceof IllegalStateException); + assertEquals("getPendingTxCommitQueueSize", 0, shard.underlyingActor().getPendingTxCommitQueueSize()); }}; } @@ -2062,7 +1839,6 @@ public class ShardTest extends AbstractShardTest { testCreateSnapshot(false, "testCreateSnapshotWithNonPersistentData"); } - @SuppressWarnings("serial") private void testCreateSnapshot(final boolean persistent, final String shardActorName) throws Exception{ final AtomicReference latch = new AtomicReference<>(new CountDownLatch(1)); @@ -2085,7 +1861,7 @@ public class ShardTest extends AbstractShardTest { new ShardTestKit(getSystem()) {{ class TestShard extends Shard { - protected TestShard(AbstractBuilder builder) { + protected TestShard(final AbstractBuilder builder) { super(builder); setPersistence(new TestPersistentDataProvider(super.persistence())); } @@ -2094,7 +1870,8 @@ public class ShardTest extends AbstractShardTest { public void handleCommand(final Object message) { super.handleCommand(message); - if (message instanceof SaveSnapshotSuccess || message.equals("commit_snapshot")) { + // XXX: commit_snapshot equality check references RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT + if (message instanceof SaveSnapshotSuccess || "commit_snapshot".equals(message.toString())) { latch.get().countDown(); } } @@ -2105,20 +1882,16 @@ public class ShardTest extends AbstractShardTest { } } - final Creator creator = new Creator() { - @Override - public Shard create() throws Exception { - return new TestShard(newShardBuilder()); - } - }; + final Creator creator = () -> new TestShard(newShardBuilder()); final TestActorRef shard = actorFactory.createTestActor( - Props.create(new DelegatingShardCreator(creator)), shardActorName); + Props.create(new DelegatingShardCreator(creator)). + withDispatcher(Dispatchers.DefaultDispatcherId()), shardActorName); waitUntilLeader(shard); writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - final NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build()); + final NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.EMPTY); // Trigger creation of a snapshot by ensuring final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext(); @@ -2129,7 +1902,7 @@ public class ShardTest extends AbstractShardTest { awaitAndValidateSnapshot(expectedRoot); } - private void awaitAndValidateSnapshot(NormalizedNode expectedRoot) throws InterruptedException { + private void awaitAndValidateSnapshot(final NormalizedNode expectedRoot) throws InterruptedException, IOException { assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS)); assertTrue("Invalid saved snapshot " + savedSnapshot.get(), @@ -2141,11 +1914,9 @@ public class ShardTest extends AbstractShardTest { savedSnapshot.set(null); } - private void verifySnapshot(final Snapshot snapshot, final NormalizedNode expectedRoot) { - - final NormalizedNode actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState()); + private void verifySnapshot(final Snapshot snapshot, final NormalizedNode expectedRoot) throws IOException { + final NormalizedNode actual = ShardDataTreeSnapshot.deserialize(snapshot.getState()).getRootNode().get(); assertEquals("Root node", expectedRoot, actual); - }}; } @@ -2165,16 +1936,16 @@ public class ShardTest extends AbstractShardTest { commitTransaction(store, putTransaction); - final NormalizedNode expected = readStore(store, YangInstanceIdentifier.builder().build()); + final NormalizedNode expected = readStore(store, YangInstanceIdentifier.EMPTY); final DataTreeModification writeTransaction = store.takeSnapshot().newModification(); - writeTransaction.delete(YangInstanceIdentifier.builder().build()); - writeTransaction.write(YangInstanceIdentifier.builder().build(), expected); + writeTransaction.delete(YangInstanceIdentifier.EMPTY); + writeTransaction.write(YangInstanceIdentifier.EMPTY, expected); commitTransaction(store, writeTransaction); - final NormalizedNode actual = readStore(store, YangInstanceIdentifier.builder().build()); + final NormalizedNode actual = readStore(store, YangInstanceIdentifier.EMPTY); assertEquals(expected, actual); } @@ -2273,11 +2044,11 @@ public class ShardTest extends AbstractShardTest { newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testFollowerInitialSyncStatus"); - shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational")); + shard.underlyingActor().handleNonRaftCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational")); assertEquals(false, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus()); - shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational")); + shard.underlyingActor().handleNonRaftCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational")); assertEquals(true, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus()); } @@ -2319,11 +2090,11 @@ public class ShardTest extends AbstractShardTest { public void testClusteredDataChangeListenerRegistration() throws Exception { new ShardTestKit(getSystem()) {{ String testName = "testClusteredDataChangeListenerRegistration"; - final ShardIdentifier followerShardID = ShardIdentifier.builder().memberName( - actorFactory.generateActorId(testName + "-follower")).shardName("inventory").type("config").build(); + final ShardIdentifier followerShardID = ShardIdentifier.create("inventory", + MemberName.forName(actorFactory.generateActorId(testName + "-follower")), "config"); - final ShardIdentifier leaderShardID = ShardIdentifier.builder().memberName( - actorFactory.generateActorId(testName + "-leader")).shardName("inventory").type("config").build(); + final ShardIdentifier leaderShardID = ShardIdentifier.create("inventory", + MemberName.forName(actorFactory.generateActorId(testName + "-leader")), "config"); final TestActorRef followerShard = actorFactory.createTestActor( Shard.builder().id(followerShardID). @@ -2338,7 +2109,7 @@ public class ShardTest extends AbstractShardTest { "akka://test/user/" + followerShardID.toString())).schemaContext(SCHEMA_CONTEXT).props(). withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString()); - leaderShard.tell(new ElectionTimeout(), ActorRef.noSender()); + leaderShard.tell(TimeoutNow.INSTANCE, ActorRef.noSender()); String leaderPath = waitUntilLeader(followerShard); assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath); @@ -2377,8 +2148,6 @@ public class ShardTest extends AbstractShardTest { waitUntilNoLeader(shard); - final YangInstanceIdentifier path = TestModel.TEST_PATH; - shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef()); final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"), RegisterDataTreeChangeListenerReply.class); @@ -2395,11 +2164,11 @@ public class ShardTest extends AbstractShardTest { public void testClusteredDataTreeChangeListenerRegistration() throws Exception { new ShardTestKit(getSystem()) {{ String testName = "testClusteredDataTreeChangeListenerRegistration"; - final ShardIdentifier followerShardID = ShardIdentifier.builder().memberName( - actorFactory.generateActorId(testName + "-follower")).shardName("inventory").type("config").build(); + final ShardIdentifier followerShardID = ShardIdentifier.create("inventory", + MemberName.forName(actorFactory.generateActorId(testName + "-follower")), "config"); - final ShardIdentifier leaderShardID = ShardIdentifier.builder().memberName( - actorFactory.generateActorId(testName + "-leader")).shardName("inventory").type("config").build(); + final ShardIdentifier leaderShardID = ShardIdentifier.create("inventory", + MemberName.forName(actorFactory.generateActorId(testName + "-leader")), "config"); final TestActorRef followerShard = actorFactory.createTestActor( Shard.builder().id(followerShardID). @@ -2414,7 +2183,7 @@ public class ShardTest extends AbstractShardTest { "akka://test/user/" + followerShardID.toString())).schemaContext(SCHEMA_CONTEXT).props(). withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString()); - leaderShard.tell(new ElectionTimeout(), ActorRef.noSender()); + leaderShard.tell(TimeoutNow.INSTANCE, ActorRef.noSender()); String leaderPath = waitUntilLeader(followerShard); assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);