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%2FDistributedDataStoreRemotingIntegrationTest.java;h=7474690bcac940661552cefe0df4f3f5b10e0828;hp=9c1103e60b2e4b8a641bdde277ae1e7aaf19f583;hb=5a0edd493bafc365647bc6311b4b7da86a78645d;hpb=40460ae356add6bd8d28a25cf8b287c9bfa38b38 diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreRemotingIntegrationTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreRemotingIntegrationTest.java index 9c1103e60b..7474690bca 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreRemotingIntegrationTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreRemotingIntegrationTest.java @@ -15,6 +15,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.verify; + import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.ActorSystem; @@ -27,6 +28,7 @@ import akka.pattern.Patterns; import akka.testkit.JavaTestKit; import com.google.common.base.Optional; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; @@ -41,14 +43,12 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier; +import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker; import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder; -import org.opendaylight.controller.cluster.datastore.IntegrationTestKit.ShardStatsVerifier; -import org.opendaylight.controller.cluster.datastore.MemberNode.RaftStateVerifier; import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException; import org.opendaylight.controller.cluster.datastore.exceptions.ShardLeaderNotRespondingException; -import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats; import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction; import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree; @@ -56,9 +56,8 @@ import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransact import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply; import org.opendaylight.controller.cluster.datastore.modification.MergeModification; import org.opendaylight.controller.cluster.datastore.modification.WriteModification; -import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries; -import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState; import org.opendaylight.controller.cluster.raft.client.messages.Shutdown; +import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries; import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy; import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal; import org.opendaylight.controller.md.cluster.datastore.model.CarsModel; @@ -98,13 +97,15 @@ import scala.concurrent.duration.FiniteDuration; * * @author Thomas Pantelis */ -public class DistributedDataStoreRemotingIntegrationTest { +public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { private static final String[] CARS_AND_PEOPLE = {"cars", "people"}; private static final String[] CARS = {"cars"}; - private static final Address MEMBER_1_ADDRESS = AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2558"); - private static final Address MEMBER_2_ADDRESS = AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2559"); + private static final Address MEMBER_1_ADDRESS = AddressFromURIString.parse( + "akka.tcp://cluster-test@127.0.0.1:2558"); + private static final Address MEMBER_2_ADDRESS = AddressFromURIString.parse( + "akka.tcp://cluster-test@127.0.0.1:2559"); private static final String MODULE_SHARDS_CARS_ONLY_1_2 = "module-shards-cars-member-1-and-2.conf"; private static final String MODULE_SHARDS_CARS_PEOPLE_1_2 = "module-shards-member1-and-2.conf"; @@ -118,11 +119,13 @@ public class DistributedDataStoreRemotingIntegrationTest { DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(2); private final DatastoreContext.Builder followerDatastoreContextBuilder = - DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5). - customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()); + DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5) + .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()); + private final TransactionIdentifier tx1 = nextTransactionId(); + private final TransactionIdentifier tx2 = nextTransactionId(); - private DistributedDataStore followerDistributedDataStore; - private DistributedDataStore leaderDistributedDataStore; + private AbstractDataStore followerDistributedDataStore; + private AbstractDataStore leaderDistributedDataStore; private IntegrationTestKit followerTestKit; private IntegrationTestKit leaderTestKit; @@ -140,50 +143,60 @@ public class DistributedDataStoreRemotingIntegrationTest { @After public void tearDown() { + if (followerDistributedDataStore != null) { + leaderDistributedDataStore.close(); + } + if (leaderDistributedDataStore != null) { + leaderDistributedDataStore.close(); + } + JavaTestKit.shutdownActorSystem(leaderSystem); JavaTestKit.shutdownActorSystem(followerSystem); JavaTestKit.shutdownActorSystem(follower2System); } - private void initDatastoresWithCars(String type) { + private void initDatastoresWithCars(final String type) { initDatastores(type, MODULE_SHARDS_CARS_ONLY_1_2, CARS); } - private void initDatastoresWithCarsAndPeople(String type) { + private void initDatastoresWithCarsAndPeople(final String type) { initDatastores(type, MODULE_SHARDS_CARS_PEOPLE_1_2, CARS_AND_PEOPLE); } - private void initDatastores(String type, String moduleShardsConfig, String[] shards) { + private void initDatastores(final String type, final String moduleShardsConfig, final String[] shards) { leaderTestKit = new IntegrationTestKit(leaderSystem, leaderDatastoreContextBuilder); leaderDistributedDataStore = leaderTestKit.setupDistributedDataStore(type, moduleShardsConfig, false, shards); followerTestKit = new IntegrationTestKit(followerSystem, followerDatastoreContextBuilder); - followerDistributedDataStore = followerTestKit.setupDistributedDataStore(type, moduleShardsConfig, false, shards); + followerDistributedDataStore = followerTestKit.setupDistributedDataStore(type, moduleShardsConfig, false, + shards); leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(), shards); } - private static void verifyCars(DOMStoreReadTransaction readTx, MapEntryNode... entries) throws Exception { + private static void verifyCars(final DOMStoreReadTransaction readTx, final MapEntryNode... entries) + throws Exception { Optional> optional = readTx.read(CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS); assertEquals("isPresent", true, optional.isPresent()); CollectionNodeBuilder listBuilder = ImmutableNodes.mapNodeBuilder(CarsModel.CAR_QNAME); - for(NormalizedNode entry: entries) { + for (NormalizedNode entry: entries) { listBuilder.withChild((MapEntryNode) entry); } assertEquals("Car list node", listBuilder.build(), optional.get()); } - private static void verifyNode(DOMStoreReadTransaction readTx, YangInstanceIdentifier path, NormalizedNode expNode) - throws Exception { + private static void verifyNode(final DOMStoreReadTransaction readTx, final YangInstanceIdentifier path, + final NormalizedNode expNode) throws Exception { Optional> optional = readTx.read(path).get(5, TimeUnit.SECONDS); assertEquals("isPresent", true, optional.isPresent()); assertEquals("Data node", expNode, optional.get()); } - private static void verifyExists(DOMStoreReadTransaction readTx, YangInstanceIdentifier path) throws Exception { + private static void verifyExists(final DOMStoreReadTransaction readTx, final YangInstanceIdentifier path) + throws Exception { Boolean exists = readTx.exists(path).get(5, TimeUnit.SECONDS); assertEquals("exists", true, exists); } @@ -239,10 +252,10 @@ public class DistributedDataStoreRemotingIntegrationTest { ActorSystem newSystem = ActorSystem.create("reinstated-member2", ConfigFactory.load().getConfig("Member2")); - DistributedDataStore member2Datastore = new IntegrationTestKit(newSystem, leaderDatastoreContextBuilder). - setupDistributedDataStore(testName, "module-shards-member2", true, CARS_AND_PEOPLE); - - verifyCars(member2Datastore.newReadOnlyTransaction(), car2); + try (AbstractDataStore member2Datastore = new IntegrationTestKit(newSystem, leaderDatastoreContextBuilder) + .setupDistributedDataStore(testName, "module-shards-member2", true, CARS_AND_PEOPLE)) { + verifyCars(member2Datastore.newReadOnlyTransaction(), car2); + } JavaTestKit.shutdownActorSystem(newSystem); } @@ -367,7 +380,7 @@ public class DistributedDataStoreRemotingIntegrationTest { } @Test - public void testTransactionChainWithMultipleShards() throws Exception{ + public void testTransactionChainWithMultipleShards() throws Exception { initDatastoresWithCarsAndPeople("testTransactionChainWithMultipleShards"); DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain(); @@ -436,8 +449,8 @@ public class DistributedDataStoreRemotingIntegrationTest { DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction(); ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier( - new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)). - withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build(); + new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)) + .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build(); writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData); @@ -471,8 +484,8 @@ public class DistributedDataStoreRemotingIntegrationTest { writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH, PeopleModel.emptyContainer()); ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier( - new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)). - withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build(); + new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)) + .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build(); // Note that merge will validate the data and fail but put succeeds b/c deep validation is not // done for put for performance reasons. @@ -512,34 +525,38 @@ public class DistributedDataStoreRemotingIntegrationTest { // Switch the leader to the follower - sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder. - shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null)); + sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder + .shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null)); JavaTestKit.shutdownActorSystem(leaderSystem, null, true); + Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS); followerTestKit.waitUntilNoLeader(followerDistributedDataStore.getActorContext(), CARS); leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1")); Cluster.get(leaderSystem).join(MEMBER_2_ADDRESS); - DatastoreContext.Builder newMember1Builder = DatastoreContext.newBuilder(). - shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5); + DatastoreContext.Builder newMember1Builder = DatastoreContext.newBuilder() + .shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5); IntegrationTestKit newMember1TestKit = new IntegrationTestKit(leaderSystem, newMember1Builder); - newMember1TestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CARS_ONLY_1_2, false, CARS); - followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), CARS); + try (AbstractDataStore ds = + newMember1TestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CARS_ONLY_1_2, false, CARS)) { - // Write a car entry to the new leader - should switch to local Tx + followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), CARS); - writeTx = followerDistributedDataStore.newWriteOnlyTransaction(); + // Write a car entry to the new leader - should switch to local Tx - MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000)); - YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima"); - writeTx.merge(car1Path, car1); + writeTx = followerDistributedDataStore.newWriteOnlyTransaction(); - followerTestKit.doCommit(writeTx.ready()); + MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000)); + YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima"); + writeTx.merge(car1Path, car1); - verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1); + followerTestKit.doCommit(writeTx.ready()); + + verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1); + } } @SuppressWarnings("unchecked") @@ -564,11 +581,11 @@ public class DistributedDataStoreRemotingIntegrationTest { new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification); modification.ready(); - ReadyLocalTransaction readyLocal = new ReadyLocalTransaction("tx-1" , modification, true); + ReadyLocalTransaction readyLocal = new ReadyLocalTransaction(tx1 , modification, true); carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef()); Object resp = followerTestKit.expectMsgClass(Object.class); - if(resp instanceof akka.actor.Status.Failure) { + if (resp instanceof akka.actor.Status.Failure) { throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause()); } @@ -583,11 +600,11 @@ public class DistributedDataStoreRemotingIntegrationTest { new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification); modification.ready(); - readyLocal = new ReadyLocalTransaction("tx-2" , modification, false); + readyLocal = new ReadyLocalTransaction(tx2 , modification, false); carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef()); resp = followerTestKit.expectMsgClass(Object.class); - if(resp instanceof akka.actor.Status.Failure) { + if (resp instanceof akka.actor.Status.Failure) { throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause()); } @@ -600,7 +617,7 @@ public class DistributedDataStoreRemotingIntegrationTest { Mockito.doReturn(DataStoreVersions.CURRENT_VERSION).when(versionSupplier).get(); ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy( leaderDistributedDataStore.getActorContext(), Arrays.asList( - new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), "tx-2"); + new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), tx2); cohort.canCommit().get(5, TimeUnit.SECONDS); cohort.preCommit().get(5, TimeUnit.SECONDS); cohort.commit().get(5, TimeUnit.SECONDS); @@ -629,13 +646,13 @@ public class DistributedDataStoreRemotingIntegrationTest { MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000)); new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification); - ForwardedReadyTransaction forwardedReady = new ForwardedReadyTransaction("tx-1", + ForwardedReadyTransaction forwardedReady = new ForwardedReadyTransaction(tx1, DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction( - Mockito.mock(ShardDataTreeTransactionParent.class), "tx-1", modification), true); + Mockito.mock(ShardDataTreeTransactionParent.class), tx1, modification), true); carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef()); Object resp = followerTestKit.expectMsgClass(Object.class); - if(resp instanceof akka.actor.Status.Failure) { + if (resp instanceof akka.actor.Status.Failure) { throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause()); } @@ -649,13 +666,13 @@ public class DistributedDataStoreRemotingIntegrationTest { MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000)); new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification); - forwardedReady = new ForwardedReadyTransaction("tx-2", + forwardedReady = new ForwardedReadyTransaction(tx2, DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction( - Mockito.mock(ShardDataTreeTransactionParent.class), "tx-2", modification), false); + Mockito.mock(ShardDataTreeTransactionParent.class), tx2, modification), false); carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef()); resp = followerTestKit.expectMsgClass(Object.class); - if(resp instanceof akka.actor.Status.Failure) { + if (resp instanceof akka.actor.Status.Failure) { throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause()); } @@ -668,7 +685,7 @@ public class DistributedDataStoreRemotingIntegrationTest { Mockito.doReturn(DataStoreVersions.CURRENT_VERSION).when(versionSupplier).get(); ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy( leaderDistributedDataStore.getActorContext(), Arrays.asList( - new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), "tx-2"); + new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), tx2); cohort.canCommit().get(5, TimeUnit.SECONDS); cohort.preCommit().get(5, TimeUnit.SECONDS); cohort.commit().get(5, TimeUnit.SECONDS); @@ -691,12 +708,11 @@ public class DistributedDataStoreRemotingIntegrationTest { // Wait for the commit to be replicated to the follower. - MemberNode.verifyRaftState(followerDistributedDataStore, "cars", new RaftStateVerifier() { - @Override - public void verify(OnDemandRaftState raftState) { - assertEquals("getLastApplied", 0, raftState.getLastApplied()); - } - }); + MemberNode.verifyRaftState(followerDistributedDataStore, "cars", + raftState -> assertEquals("getLastApplied", 0, raftState.getLastApplied())); + + MemberNode.verifyRaftState(followerDistributedDataStore, "people", + raftState -> assertEquals("getLastApplied", 0, raftState.getLastApplied())); // Prepare, ready and canCommit a WO tx that writes to 2 shards. This will become the current tx in // the leader shard. @@ -719,7 +735,7 @@ public class DistributedDataStoreRemotingIntegrationTest { carIndex++; NormalizedNode people = PeopleModel.newPersonMapNode(); writeTx2.write(PeopleModel.PERSON_LIST_PATH, people); - DOMStoreThreePhaseCommitCohort writeTx2Cohort = writeTx2.ready(); + final DOMStoreThreePhaseCommitCohort writeTx2Cohort = writeTx2.ready(); // Prepare another WO that writes to a single shard and thus will be directly committed on ready. This // tx writes 5 cars so 2 BatchedModidifications messages will be sent initially and cached in the @@ -727,7 +743,7 @@ public class DistributedDataStoreRemotingIntegrationTest { // sent on ready. DOMStoreWriteTransaction writeTx3 = followerDistributedDataStore.newWriteOnlyTransaction(); - for(int i = 1; i <= 5; i++, carIndex++) { + for (int i = 1; i <= 5; i++, carIndex++) { cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex))); writeTx3.write(CarsModel.newCarPath("car" + carIndex), cars.getLast()); } @@ -747,33 +763,30 @@ public class DistributedDataStoreRemotingIntegrationTest { cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex))); readWriteTx.write(CarsModel.newCarPath("car" + carIndex), cars.getLast()); - IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", new ShardStatsVerifier() { - @Override - public void verify(ShardStats stats) { - assertEquals("getReadWriteTransactionCount", 1, stats.getReadWriteTransactionCount()); - } - }); + IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", + stats -> assertEquals("getReadWriteTransactionCount", 1, stats.getReadWriteTransactionCount())); // Disable elections on the leader so it switches to follower. - sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder. - customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()). - shardElectionTimeoutFactor(10)); + sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder + .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()) + .shardElectionTimeoutFactor(10)); + Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS); leaderTestKit.waitUntilNoLeader(leaderDistributedDataStore.getActorContext(), "cars"); // Submit all tx's - the messages should get queued for retry. ListenableFuture writeTx2CanCommit = writeTx2Cohort.canCommit(); - DOMStoreThreePhaseCommitCohort writeTx3Cohort = writeTx3.ready(); - DOMStoreThreePhaseCommitCohort writeTx4Cohort = writeTx4.ready(); - DOMStoreThreePhaseCommitCohort rwTxCohort = readWriteTx.ready(); + final DOMStoreThreePhaseCommitCohort writeTx3Cohort = writeTx3.ready(); + final DOMStoreThreePhaseCommitCohort writeTx4Cohort = writeTx4.ready(); + final DOMStoreThreePhaseCommitCohort rwTxCohort = readWriteTx.ready(); // Enable elections on the other follower so it becomes the leader, at which point the // tx's should get forwarded from the previous leader to the new leader to complete the commits. - sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder. - customRaftPolicyImplementation(null).shardElectionTimeoutFactor(1)); + sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder + .customRaftPolicyImplementation(null).shardElectionTimeoutFactor(1)); followerTestKit.doCommit(writeTx1CanCommit, writeTx1Cohort); followerTestKit.doCommit(writeTx2CanCommit, writeTx2Cohort); @@ -794,106 +807,111 @@ public class DistributedDataStoreRemotingIntegrationTest { initDatastores(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, CARS_AND_PEOPLE); IntegrationTestKit follower2TestKit = new IntegrationTestKit(follower2System, followerDatastoreContextBuilder); - DistributedDataStore follower2DistributedDataStore = follower2TestKit.setupDistributedDataStore(testName, - MODULE_SHARDS_CARS_PEOPLE_1_2_3, false); + try (AbstractDataStore follower2DistributedDataStore = follower2TestKit.setupDistributedDataStore(testName, + MODULE_SHARDS_CARS_PEOPLE_1_2_3, false)) { - // Create and submit a couple tx's so they're pending. + // Create and submit a couple tx's so they're pending. - DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction(); - writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer()); - writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()); - writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer()); - DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready(); + DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction(); + writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer()); + writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()); + writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer()); + final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready(); - IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", new ShardStatsVerifier() { - @Override - public void verify(ShardStats stats) { - assertEquals("getTxCohortCacheSize", 1, stats.getTxCohortCacheSize()); - } - }); + IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", + stats -> assertEquals("getTxCohortCacheSize", 1, stats.getTxCohortCacheSize())); - writeTx = followerDistributedDataStore.newWriteOnlyTransaction(); - MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000)); - writeTx.write(CarsModel.newCarPath("optima"), car); - DOMStoreThreePhaseCommitCohort cohort2 = writeTx.ready(); + writeTx = followerDistributedDataStore.newWriteOnlyTransaction(); + MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000)); + writeTx.write(CarsModel.newCarPath("optima"), car); + final DOMStoreThreePhaseCommitCohort cohort2 = writeTx.ready(); - IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", new ShardStatsVerifier() { - @Override - public void verify(ShardStats stats) { - assertEquals("getTxCohortCacheSize", 2, stats.getTxCohortCacheSize()); - } - }); + IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", + stats -> assertEquals("getTxCohortCacheSize", 2, stats.getTxCohortCacheSize())); - // Gracefully stop the leader via a Shutdown message. + // Gracefully stop the leader via a Shutdown message. - sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder. - shardElectionTimeoutFactor(100)); + sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder + .shardElectionTimeoutFactor(100)); - FiniteDuration duration = FiniteDuration.create(5, TimeUnit.SECONDS); - Future future = leaderDistributedDataStore.getActorContext().findLocalShardAsync("cars"); - ActorRef leaderActor = Await.result(future, duration); + FiniteDuration duration = FiniteDuration.create(5, TimeUnit.SECONDS); + Future future = leaderDistributedDataStore.getActorContext().findLocalShardAsync("cars"); + ActorRef leaderActor = Await.result(future, duration); - Future stopFuture = Patterns.gracefulStop(leaderActor, duration, Shutdown.INSTANCE); + Future stopFuture = Patterns.gracefulStop(leaderActor, duration, Shutdown.INSTANCE); - // Commit the 2 transactions. They should finish and succeed. + // Commit the 2 transactions. They should finish and succeed. - followerTestKit.doCommit(cohort1); - followerTestKit.doCommit(cohort2); + followerTestKit.doCommit(cohort1); + followerTestKit.doCommit(cohort2); - // Wait for the leader actor stopped. + // Wait for the leader actor stopped. - Boolean stopped = Await.result(stopFuture, duration); - assertEquals("Stopped", Boolean.TRUE, stopped); + Boolean stopped = Await.result(stopFuture, duration); + assertEquals("Stopped", Boolean.TRUE, stopped); - // Verify leadership was transferred by reading the committed data from the other nodes. + // Verify leadership was transferred by reading the committed data from the other nodes. - verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car); - verifyCars(follower2DistributedDataStore.newReadOnlyTransaction(), car); + verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car); + verifyCars(follower2DistributedDataStore.newReadOnlyTransaction(), car); + } } @Test - public void testTransactionWithIsolatedLeader() throws Throwable { - leaderDatastoreContextBuilder.shardIsolatedLeaderCheckIntervalInMillis(200); + public void testTransactionWithIsolatedLeader() throws Exception { + // Set the isolated leader check interval high so we can control the switch to IsolatedLeader. + leaderDatastoreContextBuilder.shardIsolatedLeaderCheckIntervalInMillis(10000000); String testName = "testTransactionWithIsolatedLeader"; initDatastoresWithCars(testName); - DOMStoreWriteTransaction failWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction(); - failWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer()); + // Tx that is submitted after the follower is stopped but before the leader transitions to IsolatedLeader. + DOMStoreWriteTransaction preIsolatedLeaderWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction(); + preIsolatedLeaderWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer()); + + // Tx that is submitted after the leader transitions to IsolatedLeader. + DOMStoreWriteTransaction noShardLeaderWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction(); + noShardLeaderWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer()); + // Tx that is submitted after the follower is reinstated. DOMStoreWriteTransaction successWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction(); successWriteTx.merge(CarsModel.BASE_PATH, CarsModel.emptyContainer()); + // Stop the follower followerTestKit.watch(followerDistributedDataStore.getActorContext().getShardManager()); followerDistributedDataStore.close(); followerTestKit.expectTerminated(followerDistributedDataStore.getActorContext().getShardManager()); - MemberNode.verifyRaftState(leaderDistributedDataStore, "cars", new RaftStateVerifier() { - @Override - public void verify(OnDemandRaftState raftState) { - assertEquals("getRaftState", "IsolatedLeader", raftState.getRaftState()); - } - }); + // Submit the preIsolatedLeaderWriteTx so it's pending + final DOMStoreThreePhaseCommitCohort preIsolatedLeaderTxCohort = preIsolatedLeaderWriteTx.ready(); + + // Change the isolated leader check interval low so it changes to IsolatedLeader. + sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder + .shardIsolatedLeaderCheckIntervalInMillis(200)); + + MemberNode.verifyRaftState(leaderDistributedDataStore, "cars", + raftState -> assertEquals("getRaftState", "IsolatedLeader", raftState.getRaftState())); try { - leaderTestKit.doCommit(failWriteTx.ready()); + leaderTestKit.doCommit(noShardLeaderWriteTx.ready()); fail("Expected NoShardLeaderException"); } catch (ExecutionException e) { assertEquals("getCause", NoShardLeaderException.class, e.getCause().getClass()); } - sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder. - shardElectionTimeoutFactor(100)); + sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder + .shardElectionTimeoutFactor(100)); - DOMStoreThreePhaseCommitCohort writeTxCohort = successWriteTx.ready(); + DOMStoreThreePhaseCommitCohort successTxCohort = successWriteTx.ready(); followerDistributedDataStore = followerTestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CARS_ONLY_1_2, false, CARS); - leaderTestKit.doCommit(writeTxCohort); + leaderTestKit.doCommit(preIsolatedLeaderTxCohort); + leaderTestKit.doCommit(successTxCohort); } - @Test(expected=AskTimeoutException.class) - public void testTransactionWithShardLeaderNotResponding() throws Throwable { + @Test(expected = AskTimeoutException.class) + public void testTransactionWithShardLeaderNotResponding() throws Exception { initDatastoresWithCars("testTransactionWithShardLeaderNotResponding"); // Do an initial read to get the primary shard info cached. @@ -918,12 +936,13 @@ public class DistributedDataStoreRemotingIntegrationTest { assertTrue("Expected ShardLeaderNotRespondingException cause. Actual: " + e.getCause(), e.getCause() instanceof ShardLeaderNotRespondingException); assertNotNull("Expected a nested cause", e.getCause().getCause()); - throw e.getCause().getCause(); + Throwables.propagateIfInstanceOf(e.getCause().getCause(), Exception.class); + Throwables.propagate(e.getCause().getCause()); } } - @Test(expected=NoShardLeaderException.class) - public void testTransactionWithCreateTxFailureDueToNoLeader() throws Throwable { + @Test(expected = NoShardLeaderException.class) + public void testTransactionWithCreateTxFailureDueToNoLeader() throws Exception { initDatastoresWithCars("testTransactionWithCreateTxFailureDueToNoLeader"); // Do an initial read to get the primary shard info cached. @@ -935,10 +954,12 @@ public class DistributedDataStoreRemotingIntegrationTest { JavaTestKit.shutdownActorSystem(leaderSystem, null, true); + Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS); + Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); - sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder. - operationTimeoutInMillis(10).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null)); + sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder + .operationTimeoutInMillis(10).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null)); DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction(); @@ -947,7 +968,8 @@ public class DistributedDataStoreRemotingIntegrationTest { try { followerTestKit.doCommit(rwTx.ready()); } catch (ExecutionException e) { - throw e.getCause(); + Throwables.propagateIfInstanceOf(e.getCause(), Exception.class); + Throwables.propagate(e.getCause()); } } @@ -956,39 +978,42 @@ public class DistributedDataStoreRemotingIntegrationTest { String testName = "testTransactionRetryWithInitialAskTimeoutExOnCreateTx"; initDatastores(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, CARS); - DatastoreContext.Builder follower2DatastoreContextBuilder = DatastoreContext.newBuilder(). - shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5); + DatastoreContext.Builder follower2DatastoreContextBuilder = DatastoreContext.newBuilder() + .shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5); IntegrationTestKit follower2TestKit = new IntegrationTestKit(follower2System, follower2DatastoreContextBuilder); - follower2TestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, false, CARS); - // Do an initial read to get the primary shard info cached. + try (AbstractDataStore ds = + follower2TestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, false, CARS)) { - DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction(); - readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS); + followerTestKit.waitForMembersUp("member-1", "member-3"); + follower2TestKit.waitForMembersUp("member-1", "member-2"); - // Shutdown the leader and try to create a new tx. + // Do an initial read to get the primary shard info cached. - JavaTestKit.shutdownActorSystem(leaderSystem, null, true); + DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction(); + readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS); - sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder. - operationTimeoutInMillis(500).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null)); + // Shutdown the leader and try to create a new tx. - DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction(); + JavaTestKit.shutdownActorSystem(leaderSystem, null, true); - rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer()); + Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS); - followerTestKit.doCommit(rwTx.ready()); + sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder + .operationTimeoutInMillis(500).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null)); + + DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction(); + + rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer()); + + followerTestKit.doCommit(rwTx.ready()); + } } - private static void sendDatastoreContextUpdate(DistributedDataStore dataStore, final Builder builder) { + private static void sendDatastoreContextUpdate(final AbstractDataStore dataStore, final Builder builder) { final Builder newBuilder = DatastoreContext.newBuilderFrom(builder.build()); DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class); - Answer answer = new Answer() { - @Override - public DatastoreContext answer(InvocationOnMock invocation) { - return newBuilder.build(); - } - }; + Answer answer = invocation -> newBuilder.build(); Mockito.doAnswer(answer).when(mockContextFactory).getBaseDatastoreContext(); Mockito.doAnswer(answer).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString()); dataStore.onDatastoreContextUpdated(mockContextFactory);