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=f097c19e512a3766a3d836c5cc7e061862b29666;hp=24a5225b8744edb82981f8eb7dc4ab07eee4a6b4;hb=4680d02510a884b3a893345f423cedcc8c5af0f4;hpb=2088e1ffef92b11e9d877b2a51280b142a84c58b 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 24a5225b87..f097c19e51 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 @@ -1,3 +1,11 @@ +/* + * Copyright (c) 2014, 2015 Cisco Systems, Inc. and others. All rights reserved. + * + * This program and the accompanying materials are made available under the + * terms of the Eclipse Public License v1.0 which accompanies this distribution, + * and is available at http://www.eclipse.org/legal/epl-v10.html + */ + package org.opendaylight.controller.cluster.datastore; import static org.junit.Assert.assertEquals; @@ -32,7 +40,6 @@ 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; @@ -87,6 +94,7 @@ import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyn import org.opendaylight.controller.cluster.raft.client.messages.FindLeader; import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply; import org.opendaylight.controller.cluster.raft.messages.RequestVote; +import org.opendaylight.controller.cluster.raft.messages.ServerRemoved; import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal; import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore; import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor; @@ -112,10 +120,10 @@ import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidates; import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification; import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException; import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType; +import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType; import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes; import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder; import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory; -import org.opendaylight.yangtools.yang.model.api.SchemaContext; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; @@ -140,7 +148,7 @@ public class ShardTest extends AbstractShardTest { "testRegisterChangeListener-DataChangeListener"); shard.tell(new RegisterChangeListener(TestModel.TEST_PATH, - dclActor, AsyncDataBroker.DataChangeScope.BASE), getRef()); + dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef()); final RegisterChangeListenerReply reply = expectMsgClass(duration("3 seconds"), RegisterChangeListenerReply.class); @@ -182,8 +190,7 @@ public class ShardTest extends AbstractShardTest { // this will cause all other messages to not be queued properly after that. // The basic issue is that you cannot use TestActorRef with a persistent actor (at least when // it does do a persist) - return new Shard(shardID, Collections.emptyMap(), - dataStoreContextBuilder.persistent(false).build(), SCHEMA_CONTEXT) { + return new Shard(newShardBuilder()) { @Override public void onReceiveCommand(final Object message) throws Exception { if(message instanceof ElectionTimeout && firstElectionTimeout) { @@ -220,7 +227,7 @@ public class ShardTest extends AbstractShardTest { "testRegisterChangeListenerWhenNotLeaderInitially-DataChangeListener"); final TestActorRef shard = TestActorRef.create(getSystem(), - Props.create(new DelegatingShardCreator(creator)), + Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "testRegisterChangeListenerWhenNotLeaderInitially"); // Write initial data into the in-memory store. @@ -233,7 +240,7 @@ public class ShardTest extends AbstractShardTest { // Now send the RegisterChangeListener and wait for the reply. shard.tell(new RegisterChangeListener(path, dclActor, - AsyncDataBroker.DataChangeScope.SUBTREE), getRef()); + AsyncDataBroker.DataChangeScope.SUBTREE, false), getRef()); final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"), RegisterChangeListenerReply.class); @@ -301,8 +308,8 @@ public class ShardTest extends AbstractShardTest { @Override public Shard create() throws Exception { - return new Shard(shardID, Collections.emptyMap(), - dataStoreContextBuilder.persistent(false).build(), SCHEMA_CONTEXT) { + return new Shard(Shard.builder().id(shardID).datastoreContext( + dataStoreContextBuilder.persistent(false).build()).schemaContext(SCHEMA_CONTEXT)) { @Override public void onReceiveCommand(final Object message) throws Exception { if(message instanceof ElectionTimeout && firstElectionTimeout) { @@ -331,18 +338,18 @@ public class ShardTest extends AbstractShardTest { "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener"); final TestActorRef shard = TestActorRef.create(getSystem(), - Props.create(new DelegatingShardCreator(creator)), + Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "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)); + onFirstElectionTimeout.await(5, TimeUnit.SECONDS)); shard.tell(new RegisterDataTreeChangeListener(path, dclActor), getRef()); final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"), - RegisterDataTreeChangeListenerReply.class); + RegisterDataTreeChangeListenerReply.class); assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath()); shard.tell(new FindLeader(), getRef()); @@ -414,12 +421,13 @@ public class ShardTest extends AbstractShardTest { final CountDownLatch recoveryComplete = new CountDownLatch(1); class TestShard extends Shard { TestShard() { - super(shardID, Collections.singletonMap(shardID.toString(), null), - newDatastoreContext(), SCHEMA_CONTEXT); + super(Shard.builder().id(shardID).datastoreContext(newDatastoreContext()). + peerAddresses(Collections.singletonMap(shardID.toString(), null)). + schemaContext(SCHEMA_CONTEXT)); } - Map getPeerAddresses() { - return getRaftActorContext().getPeerAddresses(); + String getPeerAddress(String id) { + return getRaftActorContext().getPeerAddress(id); } @Override @@ -440,15 +448,14 @@ public class ShardTest extends AbstractShardTest { } })), "testPeerAddressResolved"); - //waitUntilLeader(shard); assertEquals("Recovery complete", true, - Uninterruptibles.awaitUninterruptibly(recoveryComplete, 5, TimeUnit.SECONDS)); + Uninterruptibles.awaitUninterruptibly(recoveryComplete, 5, TimeUnit.SECONDS)); final String address = "akka://foobar"; shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID.toString(), address)); - assertEquals("getPeerAddresses", address, - ((TestShard)shard.underlyingActor()).getPeerAddresses().get(shardID.toString())); + assertEquals("getPeerAddress", address, + ((TestShard) shard.underlyingActor()).getPeerAddress(shardID.toString())); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; @@ -464,7 +471,7 @@ public class ShardTest extends AbstractShardTest { testkit.waitUntilLeader(shard); - final DataTree store = InMemoryDataTreeFactory.getInstance().create(); + final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL); store.setSchemaContext(SCHEMA_CONTEXT); final ContainerNode container = ImmutableContainerNodeBuilder.create().withNodeIdentifier( @@ -535,7 +542,7 @@ public class ShardTest extends AbstractShardTest { } DataTree setupInMemorySnapshotStore() throws DataValidationFailedException { - final DataTree testStore = InMemoryDataTreeFactory.getInstance().create(); + final DataTree testStore = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL); testStore.setSchemaContext(SCHEMA_CONTEXT); writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); @@ -543,8 +550,8 @@ public class ShardTest extends AbstractShardTest { final NormalizedNode root = readStore(testStore, YangInstanceIdentifier.builder().build()); InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create( - SerializationUtils.serializeNormalizedNode(root), - Collections.emptyList(), 0, 1, -1, -1)); + SerializationUtils.serializeNormalizedNode(root), + Collections.emptyList(), 0, 1, -1, -1)); return testStore; } @@ -586,7 +593,7 @@ public class ShardTest extends AbstractShardTest { } InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2, - new ApplyJournalEntries(nListEntries)); + new ApplyJournalEntries(nListEntries)); testRecovery(listEntryKeys); } @@ -602,8 +609,8 @@ public class ShardTest extends AbstractShardTest { InMemoryJournal.addEntry(shardID.toString(), 0, DUMMY_DATA); InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, newModificationPayload( - new WriteModification(TestModel.OUTER_LIST_PATH, - ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build())))); + new WriteModification(TestModel.OUTER_LIST_PATH, + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build())))); final int nListEntries = 16; final Set listEntryKeys = new HashSet<>(); @@ -928,12 +935,12 @@ public class ShardTest extends AbstractShardTest { // Send a couple more BatchedModifications. shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH, - ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef()); + ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef()); expectMsgClass(duration, BatchedModificationsReply.class); shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder( TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(), - ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true, true, 3), getRef()); + ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true, true, 3), getRef()); expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS); @@ -1092,8 +1099,7 @@ public class ShardTest extends AbstractShardTest { @Override public Shard create() throws Exception { - return new Shard(shardID, Collections.emptyMap(), - newDatastoreContext(), SCHEMA_CONTEXT) { + return new Shard(newShardBuilder()) { @Override protected boolean isLeader() { return overrideLeaderCalls.get() ? false : super.isLeader(); @@ -1175,7 +1181,7 @@ public class ShardTest extends AbstractShardTest { final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - final DataTreeModification modification = dataStore.getDataTree().takeSnapshot().newModification(); + final DataTreeModification modification = dataStore.newModification(); final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME); new WriteModification(TestModel.TEST_PATH, writeData).apply(modification); @@ -1208,7 +1214,7 @@ public class ShardTest extends AbstractShardTest { final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); - final DataTreeModification modification = dataStore.getDataTree().takeSnapshot().newModification(); + final DataTreeModification modification = dataStore.newModification(); final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME); new WriteModification(TestModel.TEST_PATH, writeData).apply(modification); @@ -1260,7 +1266,7 @@ public class ShardTest extends AbstractShardTest { final MutableCompositeModification modification = new MutableCompositeModification(); final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore, - TestModel.TEST_PATH, containerNode, modification); + TestModel.TEST_PATH, containerNode, modification); final FiniteDuration duration = duration("5 seconds"); @@ -1268,7 +1274,7 @@ public class ShardTest extends AbstractShardTest { // by the ShardTransaction. shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, - cohort, modification, true, false), getRef()); + cohort, modification, true, false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // Send the CanCommitTransaction message. @@ -1551,7 +1557,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef()); final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( - expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); + expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and @@ -1673,7 +1679,7 @@ public class ShardTest extends AbstractShardTest { doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION, - cohort, modification, true, false), getRef()); + cohort, modification, true, false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef()); @@ -1818,12 +1824,12 @@ public class ShardTest extends AbstractShardTest { modification, preCommit); shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, - cohort, modification, true, false), getRef()); + cohort, modification, true, false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef()); final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable( - expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); + expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS)); assertEquals("Can commit", true, canCommitReply.getCanCommit()); shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef()); @@ -2004,7 +2010,7 @@ public class ShardTest extends AbstractShardTest { TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1); shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION, - cohort1, modification1, true, false), getRef()); + cohort1, modification1, true, false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); final String transactionID2 = "tx2"; @@ -2022,7 +2028,7 @@ public class ShardTest extends AbstractShardTest { TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME), modification3); shard.tell(new ForwardedReadyTransaction(transactionID3, CURRENT_VERSION, - cohort3, modification3, true, false), getRef()); + cohort3, modification3, true, false), getRef()); expectMsgClass(duration, ReadyTransactionReply.class); // All Tx's are readied. We'll send canCommit for the last one but not the others. The others @@ -2078,7 +2084,7 @@ public class ShardTest extends AbstractShardTest { // Ready the third Tx. final String transactionID3 = "tx3"; - final DataTreeModification modification3 = dataStore.getDataTree().takeSnapshot().newModification(); + final DataTreeModification modification3 = dataStore.newModification(); new WriteModification(TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME)) .apply(modification3); modification3.ready(); @@ -2193,8 +2199,7 @@ public class ShardTest extends AbstractShardTest { final Creator creator = new Creator() { @Override public Shard create() throws Exception { - return new Shard(shardID, Collections.emptyMap(), - dataStoreContextBuilder.build(), SCHEMA_CONTEXT) { + return new Shard(newShardBuilder()) { @Override public void onReceiveCommand(final Object message) throws Exception { super.onReceiveCommand(message); @@ -2289,9 +2294,8 @@ public class ShardTest extends AbstractShardTest { new ShardTestKit(getSystem()) {{ class TestShard extends Shard { - protected TestShard(final ShardIdentifier name, final Map peerAddresses, - final DatastoreContext datastoreContext, final SchemaContext schemaContext) { - super(name, peerAddresses, datastoreContext, schemaContext); + protected TestShard(AbstractBuilder builder) { + super(builder); setPersistence(new TestPersistentDataProvider(super.persistence())); } @@ -2313,8 +2317,7 @@ public class ShardTest extends AbstractShardTest { final Creator creator = new Creator() { @Override public Shard create() throws Exception { - return new TestShard(shardID, Collections.emptyMap(), - newDatastoreContext(), SCHEMA_CONTEXT); + return new TestShard(newShardBuilder()); } }; @@ -2322,7 +2325,6 @@ public class ShardTest extends AbstractShardTest { Props.create(new DelegatingShardCreator(creator)), shardActorName); waitUntilLeader(shard); - writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); final NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build()); @@ -2330,35 +2332,35 @@ public class ShardTest extends AbstractShardTest { // Trigger creation of a snapshot by ensuring final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext(); raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1); - - assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS)); - - assertTrue("Invalid saved snapshot " + savedSnapshot.get(), - savedSnapshot.get() instanceof Snapshot); - - verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot); - - latch.set(new CountDownLatch(1)); - savedSnapshot.set(null); + awaitAndValidateSnapshot(expectedRoot); raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1); + awaitAndValidateSnapshot(expectedRoot); - assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS)); + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + } - assertTrue("Invalid saved snapshot " + savedSnapshot.get(), - savedSnapshot.get() instanceof Snapshot); + private void awaitAndValidateSnapshot(NormalizedNode expectedRoot + ) throws InterruptedException { + System.out.println("Inside awaitAndValidateSnapshot {}" + savedSnapshot.get()); + assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS)); - verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot); + assertTrue("Invalid saved snapshot " + savedSnapshot.get(), + savedSnapshot.get() instanceof Snapshot); - shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); - } + verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot); - private void verifySnapshot(final Snapshot snapshot, final NormalizedNode expectedRoot) { + latch.set(new CountDownLatch(1)); + savedSnapshot.set(null); + } - final NormalizedNode actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState()); - assertEquals("Root node", expectedRoot, actual); + private void verifySnapshot(final Snapshot snapshot, final NormalizedNode expectedRoot) { - }}; + final NormalizedNode actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState()); + assertEquals("Root node", expectedRoot, actual); + + } + }; } /** @@ -2368,7 +2370,7 @@ public class ShardTest extends AbstractShardTest { */ @Test public void testInMemoryDataTreeRestore() throws ReadFailedException, DataValidationFailedException { - final DataTree store = InMemoryDataTreeFactory.getInstance().create(); + final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL); store.setSchemaContext(SCHEMA_CONTEXT); final DataTreeModification putTransaction = store.takeSnapshot().newModification(); @@ -2397,14 +2399,14 @@ public class ShardTest extends AbstractShardTest { final DatastoreContext persistentContext = DatastoreContext.newBuilder(). shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(true).build(); - final Props persistentProps = Shard.props(shardID, Collections.emptyMap(), - persistentContext, SCHEMA_CONTEXT); + final Props persistentProps = Shard.builder().id(shardID).datastoreContext(persistentContext). + schemaContext(SCHEMA_CONTEXT).props(); final DatastoreContext nonPersistentContext = DatastoreContext.newBuilder(). shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(false).build(); - final Props nonPersistentProps = Shard.props(shardID, Collections.emptyMap(), - nonPersistentContext, SCHEMA_CONTEXT); + final Props nonPersistentProps = Shard.builder().id(shardID).datastoreContext(nonPersistentContext). + schemaContext(SCHEMA_CONTEXT).props(); new ShardTestKit(getSystem()) {{ final TestActorRef shard1 = TestActorRef.create(getSystem(), @@ -2440,12 +2442,12 @@ public class ShardTest extends AbstractShardTest { shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender()); assertEquals("isRecoveryApplicable", false, - shard.underlyingActor().persistence().isRecoveryApplicable()); + shard.underlyingActor().persistence().isRecoveryApplicable()); shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender()); assertEquals("isRecoveryApplicable", true, - shard.underlyingActor().persistence().isRecoveryApplicable()); + shard.underlyingActor().persistence().isRecoveryApplicable()); shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); }}; @@ -2469,11 +2471,11 @@ public class ShardTest extends AbstractShardTest { MessageCollectorActor.expectFirstMatching(listener, RegisterRoleChangeListenerReply.class); ShardLeaderStateChanged leaderStateChanged = MessageCollectorActor.expectFirstMatching(listener, - ShardLeaderStateChanged.class); + ShardLeaderStateChanged.class); assertEquals("getLocalShardDataTree present", true, leaderStateChanged.getLocalShardDataTree().isPresent()); assertSame("getLocalShardDataTree", shard.underlyingActor().getDataStore().getDataTree(), - leaderStateChanged.getLocalShardDataTree().get()); + leaderStateChanged.getLocalShardDataTree().get()); MessageCollectorActor.clearMessages(listener); @@ -2513,4 +2515,163 @@ public class ShardTest extends AbstractShardTest { store.validate(modification); store.commit(store.prepare(modification)); } + + @Test + public void testClusteredDataChangeListernerDelayedRegistration() throws Exception { + new ShardTestKit(getSystem()) {{ + dataStoreContextBuilder.persistent(false); + final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1); + final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1); + final Creator creator = new Creator() { + private static final long serialVersionUID = 1L; + boolean firstElectionTimeout = true; + + @Override + public Shard create() throws Exception { + return new Shard(newShardBuilder()) { + @Override + public void onReceiveCommand(final Object message) throws Exception { + if(message instanceof ElectionTimeout && firstElectionTimeout) { + firstElectionTimeout = false; + final ActorRef self = getSelf(); + new Thread() { + @Override + public void run() { + Uninterruptibles.awaitUninterruptibly( + onChangeListenerRegistered, 5, TimeUnit.SECONDS); + self.tell(message, self); + } + }.start(); + + onFirstElectionTimeout.countDown(); + } else { + super.onReceiveCommand(message); + } + } + }; + } + }; + + final MockDataChangeListener listener = new MockDataChangeListener(1); + final ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener), + "testDataChangeListenerOnFollower-DataChangeListener"); + + final TestActorRef shard = TestActorRef.create(getSystem(), + Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()). + withDispatcher(Dispatchers.DefaultDispatcherId()),"testDataChangeListenerOnFollower"); + + assertEquals("Got first ElectionTimeout", true, + onFirstElectionTimeout.await(5, TimeUnit.SECONDS)); + + shard.tell(new FindLeader(), getRef()); + final FindLeaderReply findLeadeReply = + expectMsgClass(duration("5 seconds"), FindLeaderReply.class); + assertNull("Expected the shard not to be the leader", findLeadeReply.getLeaderActor()); + + final YangInstanceIdentifier path = TestModel.TEST_PATH; + + shard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef()); + final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"), + RegisterChangeListenerReply.class); + assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath()); + + writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + + onChangeListenerRegistered.countDown(); + + listener.waitForChangeEvents(); + + dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender()); + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testClusteredDataChangeListernerRegistration() throws Exception { + dataStoreContextBuilder.persistent(false).build(); + new ShardTestKit(getSystem()) {{ + final ShardIdentifier member1ShardID = ShardIdentifier.builder().memberName("member-1") + .shardName("inventory").type("config").build(); + + final ShardIdentifier member2ShardID = ShardIdentifier.builder().memberName("member-2") + .shardName("inventory").type("config").build(); + final Creator followerShardCreator = new Creator() { + private static final long serialVersionUID = 1L; + + @Override + public Shard create() throws Exception { + return new Shard(Shard.builder().id(member1ShardID).datastoreContext(newDatastoreContext()). + peerAddresses(Collections.singletonMap(member2ShardID.toString(), + "akka://test/user/" + member2ShardID.toString())).schemaContext(SCHEMA_CONTEXT)) { + @Override + public void onReceiveCommand(final Object message) throws Exception { + + if(!(message instanceof ElectionTimeout)) { + super.onReceiveCommand(message); + } + } + }; + } + }; + + final Creator leaderShardCreator = new Creator() { + private static final long serialVersionUID = 1L; + + @Override + public Shard create() throws Exception { + return new Shard(Shard.builder().id(member2ShardID).datastoreContext(newDatastoreContext()). + peerAddresses(Collections.singletonMap(member1ShardID.toString(), + "akka://test/user/" + member1ShardID.toString())).schemaContext(SCHEMA_CONTEXT)) {}; + } + }; + + + final TestActorRef shard = TestActorRef.create(getSystem(), + Props.create(new DelegatingShardCreator(followerShardCreator)), + member1ShardID.toString()); + + final TestActorRef shardLeader = TestActorRef.create(getSystem(), + Props.create(new DelegatingShardCreator(leaderShardCreator)).withDispatcher(Dispatchers.DefaultDispatcherId()), + member2ShardID.toString()); + // Sleep to let election happen + Uninterruptibles.sleepUninterruptibly(2, TimeUnit.SECONDS); + + shard.tell(new FindLeader(), getRef()); + final FindLeaderReply findLeaderReply = + expectMsgClass(duration("5 seconds"), FindLeaderReply.class); + assertEquals("Shard leader does not match", shardLeader.path().toString(), findLeaderReply.getLeaderActor()); + + final YangInstanceIdentifier path = TestModel.TEST_PATH; + final MockDataChangeListener listener = new MockDataChangeListener(1); + final ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener), + "testDataChangeListenerOnFollower-DataChangeListener"); + + shard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef()); + final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"), + RegisterChangeListenerReply.class); + assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath()); + + writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + + listener.waitForChangeEvents(); + + dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender()); + shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); + }}; + } + + @Test + public void testServerRemoved() throws Exception { + final TestActorRef parent = TestActorRef.create(getSystem(), MessageCollectorActor.props()); + + final ActorRef shard = parent.underlyingActor().context().actorOf( + newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()), + "testServerRemoved"); + + shard.tell(new ServerRemoved("test"), ActorRef.noSender()); + + MessageCollectorActor.expectFirstMatching(parent, ServerRemoved.class); + + } + }