X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Ftest%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FShardTest.java;h=87cfae80820935e7109dbc1a325c088a645408b8;hb=b08cf9e64783ce7db00b6e41f2720f56b726d3b5;hp=c31acdad9361c2dd07053da257bbcf66eb11e3a5;hpb=5a4560d475f0ed328275f1a5c7a5dae292acfb02;p=controller.git 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 c31acdad93..87cfae8082 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 @@ -11,9 +11,9 @@ package org.opendaylight.controller.cluster.datastore; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; @@ -32,6 +32,7 @@ import akka.persistence.SaveSnapshotSuccess; import akka.testkit.TestActorRef; import akka.util.Timeout; import com.google.common.base.Function; +import com.google.common.base.Stopwatch; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.Uninterruptibles; @@ -46,6 +47,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; @@ -89,6 +93,8 @@ import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus; import org.opendaylight.controller.cluster.raft.client.messages.FindLeader; import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply; +import org.opendaylight.controller.cluster.raft.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; @@ -123,7 +129,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); @@ -175,7 +181,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 @@ -198,13 +204,15 @@ public class ShardTest extends AbstractShardTest { onFirstElectionTimeout.countDown(); } else { - super.onReceiveCommand(message); + super.handleCommand(message); } } }; } }; + setupInMemorySnapshotStore(); + final MockDataChangeListener listener = new MockDataChangeListener(1); final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener), "testRegisterChangeListenerWhenNotLeaderInitially-DataChangeListener"); @@ -213,9 +221,7 @@ public class ShardTest extends AbstractShardTest { Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "testRegisterChangeListenerWhenNotLeaderInitially"); - // Write initial data into the in-memory store. final YangInstanceIdentifier path = TestModel.TEST_PATH; - writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); // Wait until the shard receives the first ElectionTimeout message. assertEquals("Got first ElectionTimeout", true, @@ -230,10 +236,10 @@ public class ShardTest extends AbstractShardTest { assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath()); // Sanity check - verify the shard is not the leader yet. - shard.tell(new FindLeader(), getRef()); + shard.tell(FindLeader.INSTANCE, getRef()); final FindLeaderReply findLeadeReply = expectMsgClass(duration("5 seconds"), FindLeaderReply.class); - assertNull("Expected the shard not to be the leader", findLeadeReply.getLeaderActor()); + assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent()); // Signal the onChangeListenerRegistered latch to tell the thread above to proceed // with the election process. @@ -249,7 +255,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); @@ -285,10 +291,9 @@ public class ShardTest extends AbstractShardTest { @Override public Shard create() throws Exception { - return new Shard(Shard.builder().id(shardID).datastoreContext( - dataStoreContextBuilder.persistent(false).build()).schemaContext(SCHEMA_CONTEXT)) { + 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(); @@ -303,13 +308,15 @@ public class ShardTest extends AbstractShardTest { onFirstElectionTimeout.countDown(); } else { - super.onReceiveCommand(message); + super.handleCommand(message); } } }; } }; + setupInMemorySnapshotStore(); + final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1); final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener), "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener"); @@ -319,7 +326,6 @@ public class ShardTest extends AbstractShardTest { "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration"); final YangInstanceIdentifier path = TestModel.TEST_PATH; - writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); assertEquals("Got first ElectionTimeout", true, onFirstElectionTimeout.await(5, TimeUnit.SECONDS)); @@ -329,12 +335,11 @@ public class ShardTest extends AbstractShardTest { RegisterDataTreeChangeListenerReply.class); assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath()); - shard.tell(new FindLeader(), getRef()); + shard.tell(FindLeader.INSTANCE, getRef()); final FindLeaderReply findLeadeReply = expectMsgClass(duration("5 seconds"), FindLeaderReply.class); - assertNull("Expected the shard not to be the leader", findLeadeReply.getLeaderActor()); + assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent()); - writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); onChangeListenerRegistered.countDown(); @@ -352,7 +357,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"), @@ -360,7 +365,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:")); }}; } @@ -371,7 +376,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"), @@ -379,52 +384,24 @@ 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())); }}; } @@ -445,7 +422,7 @@ 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), @@ -460,7 +437,8 @@ public class ShardTest extends AbstractShardTest { @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); @@ -470,13 +448,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 @@ -490,7 +479,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<>(); @@ -505,8 +495,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, @@ -528,19 +519,19 @@ public class ShardTest extends AbstractShardTest { final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); 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 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(), modification2); - final String transactionID3 = "tx3"; + final TransactionIdentifier transactionID3 = nextTransactionId(); final MutableCompositeModification modification3 = new MutableCompositeModification(); final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore, YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) @@ -628,9 +619,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; } @@ -689,19 +680,16 @@ 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(); + final ShardCommitCoordinator.CohortDecorator cohortDecorator = (txID, actual) -> { + if(mockCohort.get() == null) { + mockCohort.set(createDelegatingMockCohort("cohort", actual)); } + + return mockCohort.get(); }; shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator); @@ -755,19 +743,16 @@ 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(); + final ShardCommitCoordinator.CohortDecorator cohortDecorator = (txID, actual) -> { + if(mockCohort.get() == null) { + mockCohort.set(createDelegatingMockCohort("cohort", actual)); } + + return mockCohort.get(); }; shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator); @@ -810,8 +795,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); @@ -837,20 +822,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); @@ -870,9 +855,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"); @@ -880,14 +865,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); @@ -938,13 +922,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()); @@ -963,24 +948,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", + shard.tell(prepareForwardedReadyTransaction(mock(ShardDataTreeCohort.class), txId, DataStoreVersions.CURRENT_VERSION, 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); } @@ -994,7 +984,7 @@ public class ShardTest extends AbstractShardTest { final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - final String transactionID = "tx1"; + final TransactionIdentifier transactionID = nextTransactionId(); final MutableCompositeModification modification = new MutableCompositeModification(); final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore, @@ -1034,7 +1024,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); @@ -1065,8 +1055,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()); @@ -1091,9 +1081,13 @@ public class ShardTest extends AbstractShardTest { } @Test - public void testCommitWithPersistenceDisabled() throws Throwable { + public void testReadWriteCommitWithPersistenceDisabled() throws Throwable { + testCommitWithPersistenceDisabled(true); + } + + @Test + public void testWriteOnlyCommitWithPersistenceDisabled() throws Throwable { testCommitWithPersistenceDisabled(true); - testCommitWithPersistenceDisabled(false); } private void testCommitWithPersistenceDisabled(final boolean readWrite) throws Throwable { @@ -1109,7 +1103,7 @@ public class ShardTest extends AbstractShardTest { // Setup a simulated transactions with a mock cohort. - final String transactionID = "tx"; + final TransactionIdentifier transactionID = nextTransactionId(); final MutableCompositeModification modification = new MutableCompositeModification(); final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore, @@ -1143,8 +1137,12 @@ public class ShardTest extends AbstractShardTest { } @Test - public void testCommitWhenTransactionHasNoModifications() { + public void testReadWriteCommitWhenTransactionHasNoModifications() { testCommitWhenTransactionHasNoModifications(true); + } + + @Test + public void testWriteOnlyCommitWhenTransactionHasNoModifications() { testCommitWhenTransactionHasNoModifications(false); } @@ -1159,7 +1157,7 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionID = "tx1"; + final TransactionIdentifier transactionID = nextTransactionId(); final MutableCompositeModification modification = new MutableCompositeModification(); final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); @@ -1201,8 +1199,12 @@ public class ShardTest extends AbstractShardTest { } @Test - public void testCommitWhenTransactionHasModifications() { + public void testReadWriteCommitWhenTransactionHasModifications() { testCommitWhenTransactionHasModifications(true); + } + + @Test + public void testWriteOnlyCommitWhenTransactionHasModifications() { testCommitWhenTransactionHasModifications(false); } @@ -1215,9 +1217,9 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionID = "tx1"; + final TransactionIdentifier transactionID = nextTransactionId(); final MutableCompositeModification modification = new MutableCompositeModification(); - modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build())); + modification.addModification(new DeleteModification(YangInstanceIdentifier.EMPTY)); final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit(); @@ -1274,15 +1276,15 @@ public class ShardTest extends AbstractShardTest { // Setup 2 simulated transactions with mock cohorts. The first one fails in the // commit phase. - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); 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(Futures.immediateFailedFuture(new RuntimeException("mock"))).when(cohort1).commit(); doReturn(mockCandidate("cohort1-candidate")).when(cohort1).getCandidate(); - final String transactionID2 = "tx2"; + final TransactionIdentifier transactionID2 = nextTransactionId(); final MutableCompositeModification modification2 = new MutableCompositeModification(); final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); @@ -1349,13 +1351,13 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); 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 TransactionIdentifier transactionID2 = nextTransactionId(); final MutableCompositeModification modification2 = new MutableCompositeModification(); final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); @@ -1423,7 +1425,7 @@ public class ShardTest extends AbstractShardTest { final FiniteDuration duration = duration("5 seconds"); - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); final MutableCompositeModification modification = new MutableCompositeModification(); final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit(); @@ -1440,7 +1442,7 @@ public class ShardTest extends AbstractShardTest { reset(cohort); - final String transactionID2 = "tx2"; + final TransactionIdentifier transactionID2 = nextTransactionId(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification), getRef()); @@ -1469,7 +1471,7 @@ public class ShardTest extends AbstractShardTest { final FiniteDuration duration = duration("5 seconds"); - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); final MutableCompositeModification modification = new MutableCompositeModification(); final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFuture(Boolean.FALSE)).when(cohort).canCommit(); @@ -1488,7 +1490,7 @@ public class ShardTest extends AbstractShardTest { reset(cohort); - final String transactionID2 = "tx2"; + final TransactionIdentifier transactionID2 = nextTransactionId(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification), getRef()); @@ -1517,7 +1519,7 @@ public class ShardTest extends AbstractShardTest { final FiniteDuration duration = duration("5 seconds"); - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); final MutableCompositeModification modification = new MutableCompositeModification(); final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1"); doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit(); @@ -1530,7 +1532,7 @@ public class ShardTest extends AbstractShardTest { reset(cohort); - final String transactionID2 = "tx2"; + final TransactionIdentifier transactionID2 = nextTransactionId(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); doReturn(Futures.immediateFuture(null)).when(cohort).preCommit(); doReturn(Futures.immediateFuture(null)).when(cohort).commit(); @@ -1538,6 +1540,7 @@ public class ShardTest extends AbstractShardTest { final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class); doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType(); doReturn(candidateRoot).when(candidate).getRootNode(); + doReturn(YangInstanceIdentifier.EMPTY).when(candidate).getRootPath(); doReturn(candidate).when(cohort).getCandidate(); shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef()); @@ -1562,12 +1565,12 @@ public class ShardTest extends AbstractShardTest { final FiniteDuration duration = duration("5 seconds"); - final String transactionID = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); 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()); + shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification, true), getRef()); expectMsgClass(duration, akka.actor.Status.Failure.class); @@ -1575,7 +1578,7 @@ public class ShardTest extends AbstractShardTest { reset(cohort); - final String transactionID2 = "tx2"; + final TransactionIdentifier transactionID2 = nextTransactionId(); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); doReturn(Futures.immediateFuture(null)).when(cohort).preCommit(); doReturn(Futures.immediateFuture(null)).when(cohort).commit(); @@ -1583,6 +1586,7 @@ public class ShardTest extends AbstractShardTest { final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class); doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType(); doReturn(candidateRoot).when(candidate).getRootNode(); + doReturn(YangInstanceIdentifier.EMPTY).when(candidate).getRootPath(); doReturn(candidate).when(cohort).getCandidate(); shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef()); @@ -1608,25 +1612,22 @@ public class ShardTest extends AbstractShardTest { final FiniteDuration duration = duration("5 seconds"); final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - final String transactionID = "tx1"; + final TransactionIdentifier transactionID = nextTransactionId(); 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; - } + 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(); @@ -1680,7 +1681,7 @@ public class ShardTest extends AbstractShardTest { // Create 1st Tx - will timeout - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); final MutableCompositeModification modification1 = new MutableCompositeModification(); final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) @@ -1690,7 +1691,7 @@ public class ShardTest extends AbstractShardTest { // Create 2nd Tx - final String transactionID2 = "tx3"; + final TransactionIdentifier transactionID2 = nextTransactionId(); final MutableCompositeModification modification2 = new MutableCompositeModification(); final YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(); @@ -1747,19 +1748,19 @@ public class ShardTest extends AbstractShardTest { final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); 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 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(), modification2); - final String transactionID3 = "tx3"; + final TransactionIdentifier transactionID3 = nextTransactionId(); final MutableCompositeModification modification3 = new MutableCompositeModification(); final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification3); @@ -1808,7 +1809,7 @@ public class ShardTest extends AbstractShardTest { final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); final MutableCompositeModification modification1 = new MutableCompositeModification(); final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1); @@ -1816,7 +1817,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - final String transactionID2 = "tx2"; + final TransactionIdentifier transactionID2 = nextTransactionId(); final MutableCompositeModification modification2 = new MutableCompositeModification(); final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2); @@ -1824,7 +1825,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); - final String transactionID3 = "tx3"; + final TransactionIdentifier transactionID3 = nextTransactionId(); final MutableCompositeModification modification3 = new MutableCompositeModification(); final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore, TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME), modification3); @@ -1855,7 +1856,7 @@ public class ShardTest extends AbstractShardTest { final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); final MutableCompositeModification modification1 = new MutableCompositeModification(); final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1); @@ -1870,7 +1871,7 @@ public class ShardTest extends AbstractShardTest { // Ready the second Tx. - final String transactionID2 = "tx2"; + final TransactionIdentifier transactionID2 = nextTransactionId(); final MutableCompositeModification modification2 = new MutableCompositeModification(); final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2); @@ -1880,7 +1881,7 @@ public class ShardTest extends AbstractShardTest { // 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); @@ -1911,7 +1912,7 @@ 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); }}; } @@ -1932,13 +1933,13 @@ public class ShardTest extends AbstractShardTest { // Setup 2 simulated transactions with mock cohorts. The first one will be aborted. - final String transactionID1 = "tx1"; + final TransactionIdentifier transactionID1 = nextTransactionId(); 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 TransactionIdentifier transactionID2 = nextTransactionId(); final MutableCompositeModification modification2 = new MutableCompositeModification(); final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2"); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit(); @@ -1992,20 +1993,15 @@ public class ShardTest extends AbstractShardTest { new ShardTestKit(getSystem()) {{ final AtomicReference cleaupCheckLatch = new AtomicReference<>(); @SuppressWarnings("serial") - final Creator creator = new Creator() { + final Creator creator = () -> new Shard(newShardBuilder()) { @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(); - } - } + public void handleCommand(final Object message) { + super.handleCommand(message); + if(TX_COMMIT_TIMEOUT_CHECK_MESSAGE.equals(message)) { + if(cleaupCheckLatch.get() != null) { + cleaupCheckLatch.get().countDown(); } - }; + } } }; @@ -2015,8 +2011,7 @@ public class ShardTest extends AbstractShardTest { waitUntilLeader(shard); - final String transactionID = "tx1"; - + final TransactionIdentifier transactionID = nextTransactionId(); final MutableCompositeModification modification = new MutableCompositeModification(); final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort"); doReturn(Futures.immediateFuture(null)).when(cohort).abort(); @@ -2064,7 +2059,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)); @@ -2096,7 +2090,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(); } } @@ -2107,20 +2102,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(); @@ -2167,16 +2158,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); } @@ -2275,11 +2266,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()); } @@ -2288,12 +2279,15 @@ public class ShardTest extends AbstractShardTest { public void testClusteredDataChangeListenerDelayedRegistration() throws Exception { new ShardTestKit(getSystem()) {{ String testName = "testClusteredDataChangeListenerDelayedRegistration"; - dataStoreContextBuilder.shardElectionTimeoutFactor(1000); + dataStoreContextBuilder.shardElectionTimeoutFactor(1000). + customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()); final MockDataChangeListener listener = new MockDataChangeListener(1); final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener), actorFactory.generateActorId(testName + "-DataChangeListener")); + setupInMemorySnapshotStore(); + final TestActorRef shard = actorFactory.createTestActor( newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()), actorFactory.generateActorId(testName + "-shard")); @@ -2307,9 +2301,8 @@ public class ShardTest extends AbstractShardTest { RegisterChangeListenerReply.class); assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath()); - writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - - shard.tell(new ElectionTimeout(), ActorRef.noSender()); + shard.tell(DatastoreContext.newBuilderFrom(dataStoreContextBuilder.build()). + customRaftPolicyImplementation(null).build(), ActorRef.noSender()); listener.waitForChangeEvents(); }}; @@ -2319,11 +2312,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 +2331,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(ElectionTimeout.INSTANCE, ActorRef.noSender()); String leaderPath = waitUntilLeader(followerShard); assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath); @@ -2362,12 +2355,15 @@ public class ShardTest extends AbstractShardTest { public void testClusteredDataTreeChangeListenerDelayedRegistration() throws Exception { new ShardTestKit(getSystem()) {{ String testName = "testClusteredDataTreeChangeListenerDelayedRegistration"; - dataStoreContextBuilder.shardElectionTimeoutFactor(1000); + dataStoreContextBuilder.shardElectionTimeoutFactor(1000). + customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()); final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1); final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener), actorFactory.generateActorId(testName + "-DataTreeChangeListener")); + setupInMemorySnapshotStore(); + final TestActorRef shard = actorFactory.createTestActor( newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()), actorFactory.generateActorId(testName + "-shard")); @@ -2381,9 +2377,8 @@ public class ShardTest extends AbstractShardTest { RegisterDataTreeChangeListenerReply.class); assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath()); - writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - - shard.tell(new ElectionTimeout(), ActorRef.noSender()); + shard.tell(DatastoreContext.newBuilderFrom(dataStoreContextBuilder.build()). + customRaftPolicyImplementation(null).build(), ActorRef.noSender()); listener.waitForChangeEvents(); }}; @@ -2393,11 +2388,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). @@ -2412,7 +2407,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(ElectionTimeout.INSTANCE, ActorRef.noSender()); String leaderPath = waitUntilLeader(followerShard); assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);