Bug 6587: Retain state when transitioning between Leader and IsolatedLeader
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreRemotingIntegrationTest.java
index f318c68c864ccada5c62f5085dee8267e63039cd..4c0aac48346a59e2db49db247cdaa37bbbd2abf0 100644 (file)
@@ -28,25 +28,25 @@ import akka.testkit.JavaTestKit;
 import com.google.common.base.Optional;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.Uninterruptibles;
 import com.typesafe.config.ConfigFactory;
 import java.math.BigInteger;
 import java.util.Arrays;
+import java.util.LinkedList;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 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;
@@ -54,9 +54,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;
@@ -96,7 +95,7 @@ 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"};
@@ -118,6 +117,8 @@ public class DistributedDataStoreRemotingIntegrationTest {
     private final DatastoreContext.Builder followerDatastoreContextBuilder =
             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;
@@ -138,6 +139,13 @@ 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);
@@ -237,10 +245,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 (DistributedDataStore member2Datastore = new IntegrationTestKit(newSystem, leaderDatastoreContextBuilder).
+                setupDistributedDataStore(testName, "module-shards-member2", true, CARS_AND_PEOPLE)) {
+            verifyCars(member2Datastore.newReadOnlyTransaction(), car2);
+        }
 
         JavaTestKit.shutdownActorSystem(newSystem);
     }
@@ -514,6 +522,7 @@ public class DistributedDataStoreRemotingIntegrationTest {
                 shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
 
         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
+        Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
 
         followerTestKit.waitUntilNoLeader(followerDistributedDataStore.getActorContext(), CARS);
 
@@ -523,21 +532,24 @@ public class DistributedDataStoreRemotingIntegrationTest {
         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 (DistributedDataStore 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);
+
+            followerTestKit.doCommit(writeTx.ready());
 
-        verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1);
+            verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1);
+        }
     }
 
     @SuppressWarnings("unchecked")
@@ -562,7 +574,7 @@ 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);
@@ -581,7 +593,7 @@ 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);
@@ -598,7 +610,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);
@@ -627,9 +639,9 @@ 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);
@@ -647,9 +659,9 @@ 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);
@@ -666,7 +678,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);
@@ -676,40 +688,76 @@ public class DistributedDataStoreRemotingIntegrationTest {
 
     @Test
     public void testTransactionForwardedToLeaderAfterRetry() throws Exception {
-        initDatastoresWithCars("testTransactionForwardedToLeaderAfterRetry");
+        followerDatastoreContextBuilder.shardBatchedModificationCount(2);
+        leaderDatastoreContextBuilder.shardBatchedModificationCount(2);
+        initDatastoresWithCarsAndPeople("testTransactionForwardedToLeaderAfterRetry");
 
         // Do an initial write to get the primary shard info cached.
 
-        DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
-        writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
-        writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
-        followerTestKit.doCommit(writeTx.ready());
+        DOMStoreWriteTransaction initialWriteTx = followerDistributedDataStore.newWriteOnlyTransaction();
+        initialWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
+        initialWriteTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
+        followerTestKit.doCommit(initialWriteTx.ready());
 
         // 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.
+
+        DOMStoreWriteTransaction writeTx1 = followerDistributedDataStore.newWriteOnlyTransaction();
+        writeTx1.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
+        writeTx1.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
+        DOMStoreThreePhaseCommitCohort writeTx1Cohort = writeTx1.ready();
+        ListenableFuture<Boolean> writeTx1CanCommit = writeTx1Cohort.canCommit();
+        writeTx1CanCommit.get(5, TimeUnit.SECONDS);
+
+        // Prepare and ready another WO tx that writes to 2 shards but don't canCommit yet. This will be queued
+        // in the leader shard.
+
+        DOMStoreWriteTransaction writeTx2 = followerDistributedDataStore.newWriteOnlyTransaction();
+        LinkedList<MapEntryNode> cars = new LinkedList<>();
+        int carIndex = 1;
+        cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
+        writeTx2.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
+        carIndex++;
+        NormalizedNode<?, ?> people = PeopleModel.newPersonMapNode();
+        writeTx2.write(PeopleModel.PERSON_LIST_PATH, people);
+        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
+        // leader shard (with shardBatchedModificationCount set to 2). The 3rd BatchedModidifications will be
+        // sent on ready.
+
+        DOMStoreWriteTransaction writeTx3 = followerDistributedDataStore.newWriteOnlyTransaction();
+        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());
+        }
 
-        // Create and prepare wo and rw tx's.
+        // Prepare another WO that writes to a single shard. This will send a single BatchedModidifications
+        // message on ready.
 
-        writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
-        MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
-        writeTx.write(CarsModel.newCarPath("optima"), car1);
+        DOMStoreWriteTransaction writeTx4 = followerDistributedDataStore.newWriteOnlyTransaction();
+        cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
+        writeTx4.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
+        carIndex++;
+
+        // Prepare a RW tx that will create a tx actor and send a ForwardedReadyTransaciton message to the
+        // leader shard on ready.
 
         DOMStoreReadWriteTransaction readWriteTx = followerDistributedDataStore.newReadWriteTransaction();
-        MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
-        readWriteTx.write(CarsModel.newCarPath("sportage"), car2);
+        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.
 
@@ -717,21 +765,31 @@ public class DistributedDataStoreRemotingIntegrationTest {
                 customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()).
                 shardElectionTimeoutFactor(10));
 
+        Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
         leaderTestKit.waitUntilNoLeader(leaderDistributedDataStore.getActorContext(), "cars");
 
-        // Submit tx's and enable elections on the follower so it becomes the leader, at which point the
-        // readied tx's should get forwarded from the previous leader.
+        // Submit all tx's - the messages should get queued for retry.
 
-        DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
-        DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
+        ListenableFuture<Boolean> writeTx2CanCommit = writeTx2Cohort.canCommit();
+        DOMStoreThreePhaseCommitCohort writeTx3Cohort = writeTx3.ready();
+        DOMStoreThreePhaseCommitCohort writeTx4Cohort = writeTx4.ready();
+        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));
 
-        followerTestKit.doCommit(cohort1);
-        followerTestKit.doCommit(cohort2);
+        followerTestKit.doCommit(writeTx1CanCommit, writeTx1Cohort);
+        followerTestKit.doCommit(writeTx2CanCommit, writeTx2Cohort);
+        followerTestKit.doCommit(writeTx3Cohort);
+        followerTestKit.doCommit(writeTx4Cohort);
+        followerTestKit.doCommit(rwTxCohort);
 
-        verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
+        DOMStoreReadTransaction readTx = leaderDistributedDataStore.newReadOnlyTransaction();
+        verifyCars(readTx, cars.toArray(new MapEntryNode[cars.size()]));
+        verifyNode(readTx, PeopleModel.PERSON_LIST_PATH, people);
     }
 
     @Test
@@ -742,85 +800,92 @@ 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 (DistributedDataStore 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());
+            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);
+            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.
 
-        FiniteDuration duration = FiniteDuration.create(5, TimeUnit.SECONDS);
-        Future<ActorRef> future = leaderDistributedDataStore.getActorContext().findLocalShardAsync("cars");
-        ActorRef leaderActor = Await.result(future, duration);
+            sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder.
+                shardElectionTimeoutFactor(100));
 
-        Future<Boolean> stopFuture = Patterns.gracefulStop(leaderActor, duration, Shutdown.INSTANCE);
+            FiniteDuration duration = FiniteDuration.create(5, TimeUnit.SECONDS);
+            Future<ActorRef> future = leaderDistributedDataStore.getActorContext().findLocalShardAsync("cars");
+            ActorRef leaderActor = Await.result(future, duration);
 
-        // Commit the 2 transactions. They should finish and succeed.
+            Future<Boolean> stopFuture = Patterns.gracefulStop(leaderActor, duration, Shutdown.INSTANCE);
 
-        followerTestKit.doCommit(cohort1);
-        followerTestKit.doCommit(cohort2);
+            // Commit the 2 transactions. They should finish and succeed.
+
+            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);
+        // 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
+        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());
@@ -829,12 +894,13 @@ public class DistributedDataStoreRemotingIntegrationTest {
         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)
@@ -880,6 +946,8 @@ public class DistributedDataStoreRemotingIntegrationTest {
 
         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
 
+        Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
+
         Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder.
@@ -904,36 +972,39 @@ public class DistributedDataStoreRemotingIntegrationTest {
         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 (DistributedDataStore 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.
+            // Shutdown the leader and try to create a new tx.
+
+            JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
+
+            Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
+
+            sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder.
                 operationTimeoutInMillis(500).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
 
-        DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
+            DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
 
-        rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
+            rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
 
-        followerTestKit.doCommit(rwTx.ready());
+            followerTestKit.doCommit(rwTx.ready());
+        }
     }
 
     private static void sendDatastoreContextUpdate(DistributedDataStore dataStore, final Builder builder) {
         final Builder newBuilder = DatastoreContext.newBuilderFrom(builder.build());
         DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
-        Answer<DatastoreContext> answer = new Answer<DatastoreContext>() {
-            @Override
-            public DatastoreContext answer(InvocationOnMock invocation) {
-                return newBuilder.build();
-            }
-        };
+        Answer<DatastoreContext> answer = invocation -> newBuilder.build();
         Mockito.doAnswer(answer).when(mockContextFactory).getBaseDatastoreContext();
         Mockito.doAnswer(answer).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
         dataStore.onDatastoreContextUpdated(mockContextFactory);