BUG-4964: Bump to akka-2.4.1
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreIntegrationTest.java
index a8384d8758a7bc3523bf75f1f3e31480d21ae737..7b78da29234706a56848031169be919a2f3458a7 100644 (file)
@@ -1,35 +1,72 @@
+/*
+ * 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;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import akka.actor.ActorRef;
+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.ActorSystem;
-import akka.actor.PoisonPill;
+import akka.actor.Address;
+import akka.actor.AddressFromURIString;
+import akka.cluster.Cluster;
+import akka.testkit.JavaTestKit;
 import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.CheckedFuture;
+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.io.IOException;
+import java.math.BigInteger;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.mockito.Mockito;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
-import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
-import org.opendaylight.controller.cluster.datastore.utils.MockClusterWrapper;
+import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot;
+import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
+import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
+import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils;
+import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
+import org.opendaylight.controller.cluster.raft.Snapshot;
 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataChangeScope;
+import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainClosedException;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
+import org.opendaylight.controller.md.sal.dom.api.DOMDataReadWriteTransaction;
+import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
+import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
+import org.opendaylight.controller.sal.core.spi.data.DOMStore;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
@@ -37,19 +74,41 @@ import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
+import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
+
+public class DistributedDataStoreIntegrationTest {
 
-public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
+    private static ActorSystem system;
 
     private final DatastoreContext.Builder datastoreContextBuilder =
             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100);
 
+    @BeforeClass
+    public static void setUpClass() throws IOException {
+        system = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
+        Address member1Address = AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2558");
+        Cluster.get(system).join(member1Address);
+    }
+
+    @AfterClass
+    public static void tearDownClass() throws IOException {
+        JavaTestKit.shutdownActorSystem(system);
+        system = null;
+    }
+
+    protected ActorSystem getSystem() {
+        return system;
+    }
+
     @Test
     public void testWriteTransactionWithSingleShard() throws Exception{
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore =
                     setupDistributedDataStore("transactionIntegrationTest", "test-1");
 
@@ -65,47 +124,59 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
 
     @Test
     public void testWriteTransactionWithMultipleShards() throws Exception{
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore =
                     setupDistributedDataStore("testWriteTransactionWithMultipleShards", "cars-1", "people-1");
 
             DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
 
-            YangInstanceIdentifier nodePath1 = CarsModel.BASE_PATH;
-            NormalizedNode<?, ?> nodeToWrite1 = CarsModel.emptyContainer();
-            writeTx.write(nodePath1, nodeToWrite1);
+            writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
+            writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
 
-            YangInstanceIdentifier nodePath2 = PeopleModel.BASE_PATH;
-            NormalizedNode<?, ?> nodeToWrite2 = PeopleModel.emptyContainer();
-            writeTx.write(nodePath2, nodeToWrite2);
+            doCommit(writeTx.ready());
 
-            DOMStoreThreePhaseCommitCohort cohort = writeTx.ready();
+            writeTx = dataStore.newWriteOnlyTransaction();
 
-            doCommit(cohort);
+            writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
+            writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
+
+            doCommit(writeTx.ready());
+
+            writeTx = dataStore.newWriteOnlyTransaction();
+
+            MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
+            YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
+            writeTx.write(carPath, car);
+
+            MapEntryNode person = PeopleModel.newPersonEntry("jack");
+            YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
+            writeTx.write(personPath, person);
+
+            doCommit(writeTx.ready());
 
             // Verify the data in the store
 
             DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
 
-            Optional<NormalizedNode<?, ?>> optional = readTx.read(nodePath1).get();
+            Optional<NormalizedNode<?, ?>> optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
             assertEquals("isPresent", true, optional.isPresent());
-            assertEquals("Data node", nodeToWrite1, optional.get());
+            assertEquals("Data node", car, optional.get());
 
-            optional = readTx.read(nodePath2).get();
+            optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
             assertEquals("isPresent", true, optional.isPresent());
-            assertEquals("Data node", nodeToWrite2, optional.get());
+            assertEquals("Data node", person, optional.get());
 
             cleanup(dataStore);
         }};
     }
 
     @Test
-    public void testReadWriteTransaction() throws Exception{
+    public void testReadWriteTransactionWithSingleShard() throws Exception{
         System.setProperty("shard.persistent", "true");
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore =
-                    setupDistributedDataStore("testReadWriteTransaction", "test-1");
+                    setupDistributedDataStore("testReadWriteTransactionWithSingleShard", "test-1");
 
             // 1. Create a read-write Tx
 
@@ -147,9 +218,97 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
         }};
     }
 
-    private void testTransactionWritesWithShardNotInitiallyReady(final boolean writeOnly) throws Exception {
-        new IntegrationTestKit(getSystem()) {{
-            String testName = "testTransactionWritesWithShardNotInitiallyReady";
+    @Test
+    public void testReadWriteTransactionWithMultipleShards() throws Exception{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            DistributedDataStore dataStore =
+                    setupDistributedDataStore("testReadWriteTransactionWithMultipleShards", "cars-1", "people-1");
+
+            DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
+            assertNotNull("newReadWriteTransaction returned null", readWriteTx);
+
+            readWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
+            readWriteTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
+
+            doCommit(readWriteTx.ready());
+
+            readWriteTx = dataStore.newReadWriteTransaction();
+
+            readWriteTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
+            readWriteTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
+
+            doCommit(readWriteTx.ready());
+
+            readWriteTx = dataStore.newReadWriteTransaction();
+
+            MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
+            YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
+            readWriteTx.write(carPath, car);
+
+            MapEntryNode person = PeopleModel.newPersonEntry("jack");
+            YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
+            readWriteTx.write(personPath, person);
+
+            Boolean exists = readWriteTx.exists(carPath).checkedGet(5, TimeUnit.SECONDS);
+            assertEquals("exists", true, exists);
+
+            Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", car, optional.get());
+
+            doCommit(readWriteTx.ready());
+
+            // Verify the data in the store
+
+            DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
+
+            optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", car, optional.get());
+
+            optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", person, optional.get());
+
+            cleanup(dataStore);
+        }};
+    }
+
+    @Test
+    public void testSingleTransactionsWritesInQuickSuccession() throws Exception{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            DistributedDataStore dataStore = setupDistributedDataStore(
+                    "testSingleTransactionsWritesInQuickSuccession", "cars-1");
+
+            DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
+
+            DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
+            writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
+            writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
+            doCommit(writeTx.ready());
+
+            writeTx = txChain.newWriteOnlyTransaction();
+
+            int nCars = 5;
+            for(int i = 0; i < nCars; i++) {
+                writeTx.write(CarsModel.newCarPath("car" + i),
+                        CarsModel.newCarEntry("car" + i, BigInteger.valueOf(20000)));
+            }
+
+            doCommit(writeTx.ready());
+
+            Optional<NormalizedNode<?, ?>> optional = txChain.newReadOnlyTransaction().read(
+                    CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("# cars", nCars, ((Collection<?>)optional.get().getValue()).size());
+
+            cleanup(dataStore);
+        }};
+    }
+
+    private void testTransactionWritesWithShardNotInitiallyReady(final String testName,
+            final boolean writeOnly) throws Exception {
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             String shardName = "test-1";
 
             // Setup the InMemoryJournal to block shard recovery to ensure the shard isn't
@@ -241,17 +400,17 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
     @Test
     public void testWriteOnlyTransactionWithShardNotInitiallyReady() throws Exception {
         datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
-        testTransactionWritesWithShardNotInitiallyReady(true);
+        testTransactionWritesWithShardNotInitiallyReady("testWriteOnlyTransactionWithShardNotInitiallyReady", true);
     }
 
     @Test
     public void testReadWriteTransactionWithShardNotInitiallyReady() throws Exception {
-        testTransactionWritesWithShardNotInitiallyReady(false);
+        testTransactionWritesWithShardNotInitiallyReady("testReadWriteTransactionWithShardNotInitiallyReady", false);
     }
 
     @Test
     public void testTransactionReadsWithShardNotInitiallyReady() throws Exception {
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             String testName = "testTransactionReadsWithShardNotInitiallyReady";
             String shardName = "test-1";
 
@@ -324,7 +483,7 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
 
     @Test(expected=NotInitializedException.class)
     public void testTransactionCommitFailureWithShardNotInitialized() throws Throwable{
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             String testName = "testTransactionCommitFailureWithShardNotInitialized";
             String shardName = "test-1";
 
@@ -338,6 +497,8 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
             CountDownLatch blockRecoveryLatch = new CountDownLatch(1);
             InMemoryJournal.addBlockReadMessagesLatch(persistentID, blockRecoveryLatch);
 
+            InMemoryJournal.addEntry(persistentID, 1, "Dummy data so akka will read from persistence");
+
             DistributedDataStore dataStore = setupDistributedDataStore(testName, false, shardName);
 
             // Create the write Tx
@@ -394,7 +555,7 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
 
     @Test(expected=NotInitializedException.class)
     public void testTransactionReadFailureWithShardNotInitialized() throws Throwable{
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             String testName = "testTransactionReadFailureWithShardNotInitialized";
             String shardName = "test-1";
 
@@ -408,6 +569,8 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
             CountDownLatch blockRecoveryLatch = new CountDownLatch(1);
             InMemoryJournal.addBlockReadMessagesLatch(persistentID, blockRecoveryLatch);
 
+            InMemoryJournal.addEntry(persistentID, 1, "Dummy data so akka will read from persistence");
+
             DistributedDataStore dataStore = setupDistributedDataStore(testName, false, shardName);
 
             // Create the read-write Tx
@@ -465,23 +628,24 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
         }};
     }
 
-    private void testTransactionCommitFailureWithNoShardLeader(final boolean writeOnly) throws Throwable {
-        new IntegrationTestKit(getSystem()) {{
-            String testName = "testTransactionCommitFailureWithNoShardLeader";
+    private void testTransactionCommitFailureWithNoShardLeader(final boolean writeOnly, final String testName) throws Throwable {
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             String shardName = "default";
 
-            // We don't want the shard to become the leader so prevent shard election from completing
-            // by setting the election timeout, which is based on the heartbeat interval, really high.
-
-            datastoreContextBuilder.shardHeartbeatIntervalInMillis(30000);
-            datastoreContextBuilder.shardInitializationTimeout(300, TimeUnit.MILLISECONDS);
+            // We don't want the shard to become the leader so prevent shard elections.
+            datastoreContextBuilder.customRaftPolicyImplementation(
+                    "org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy");
 
-            // Set the leader election timeout low for the test.
-
-            datastoreContextBuilder.shardLeaderElectionTimeout(1, TimeUnit.MILLISECONDS);
+            // The ShardManager uses the election timeout for FindPrimary so reset it low so it will timeout quickly.
+            datastoreContextBuilder.shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(1).
+                    shardInitializationTimeout(200, TimeUnit.MILLISECONDS);
 
             DistributedDataStore dataStore = setupDistributedDataStore(testName, false, shardName);
 
+            Object result = dataStore.getActorContext().executeOperation(dataStore.getActorContext().getShardManager(),
+                    new FindLocalShard(shardName, true));
+            assertTrue("Expected LocalShardFound. Actual: " + result, result instanceof LocalShardFound);
+
             // Create the write Tx.
 
             final DOMStoreWriteTransaction writeTx = writeOnly ? dataStore.newWriteOnlyTransaction() :
@@ -537,18 +701,17 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
     @Test(expected=NoShardLeaderException.class)
     public void testWriteOnlyTransactionCommitFailureWithNoShardLeader() throws Throwable {
         datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
-        testTransactionCommitFailureWithNoShardLeader(true);
+        testTransactionCommitFailureWithNoShardLeader(true, "testWriteOnlyTransactionCommitFailureWithNoShardLeader");
     }
 
     @Test(expected=NoShardLeaderException.class)
     public void testReadWriteTransactionCommitFailureWithNoShardLeader() throws Throwable {
-        testTransactionCommitFailureWithNoShardLeader(false);
+        testTransactionCommitFailureWithNoShardLeader(false, "testReadWriteTransactionCommitFailureWithNoShardLeader");
     }
 
     @Test
     public void testTransactionAbort() throws Exception{
-        System.setProperty("shard.persistent", "true");
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore =
                     setupDistributedDataStore("transactionAbortIntegrationTest", "test-1");
 
@@ -571,9 +734,9 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
     }
 
     @Test
-    public void testTransactionChain() throws Exception{
-        new IntegrationTestKit(getSystem()) {{
-            DistributedDataStore dataStore = setupDistributedDataStore("testTransactionChain", "test-1");
+    public void testTransactionChainWithSingleShard() throws Exception{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            DistributedDataStore dataStore = setupDistributedDataStore("testTransactionChainWithSingleShard", "test-1");
 
             // 1. Create a Tx chain and write-only Tx
 
@@ -658,40 +821,123 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
         }};
     }
 
+    @Test
+    public void testTransactionChainWithMultipleShards() throws Exception{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            DistributedDataStore dataStore = setupDistributedDataStore("testTransactionChainWithMultipleShards",
+                    "cars-1", "people-1");
+
+            DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
+
+            DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
+            assertNotNull("newWriteOnlyTransaction returned null", writeTx);
+
+            writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
+            writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
+
+            writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
+            writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
+
+            DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
+
+            DOMStoreReadWriteTransaction readWriteTx = txChain.newReadWriteTransaction();
+
+            MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
+            YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
+            readWriteTx.write(carPath, car);
+
+            MapEntryNode person = PeopleModel.newPersonEntry("jack");
+            YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
+            readWriteTx.merge(personPath, person);
+
+            Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", car, optional.get());
+
+            optional = readWriteTx.read(personPath).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", person, optional.get());
+
+            DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
+
+            writeTx = txChain.newWriteOnlyTransaction();
+
+            writeTx.delete(carPath);
+
+            DOMStoreThreePhaseCommitCohort cohort3 = writeTx.ready();
+
+            ListenableFuture<Boolean> canCommit1 = cohort1.canCommit();
+            ListenableFuture<Boolean> canCommit2 = cohort2.canCommit();
+
+            doCommit(canCommit1, cohort1);
+            doCommit(canCommit2, cohort2);
+            doCommit(cohort3);
+
+            txChain.close();
+
+            DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
+
+            optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", false, optional.isPresent());
+
+            optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", person, optional.get());
+
+            cleanup(dataStore);
+        }};
+    }
+
     @Test
     public void testCreateChainedTransactionsInQuickSuccession() throws Exception{
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore = setupDistributedDataStore(
-                    "testCreateChainedTransactionsInQuickSuccession", "test-1");
+                    "testCreateChainedTransactionsInQuickSuccession", "cars-1");
 
-            DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
+            ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
+                    ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
+                            LogicalDatastoreType.CONFIGURATION, dataStore).build(), MoreExecutors.directExecutor());
 
-            NormalizedNode<?, ?> testNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+            TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
+            DOMTransactionChain txChain = broker.createTransactionChain(listener);
+
+            List<CheckedFuture<Void, TransactionCommitFailedException>> futures = new ArrayList<>();
 
-            int nTxs = 20;
-            List<DOMStoreThreePhaseCommitCohort> cohorts = new ArrayList<>(nTxs);
-            for(int i = 0; i < nTxs; i++) {
-                DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
+            DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
+            writeTx.put(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, CarsModel.emptyContainer());
+            writeTx.put(LogicalDatastoreType.CONFIGURATION, CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
+            futures.add(writeTx.submit());
 
-                rwTx.merge(TestModel.TEST_PATH, testNode);
+            int nCars = 100;
+            for(int i = 0; i < nCars; i++) {
+                DOMDataReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
 
-                cohorts.add(rwTx.ready());
+                rwTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.newCarPath("car" + i),
+                        CarsModel.newCarEntry("car" + i, BigInteger.valueOf(20000)));
 
+                futures.add(rwTx.submit());
             }
 
-            for(DOMStoreThreePhaseCommitCohort cohort: cohorts) {
-                doCommit(cohort);
+            for(CheckedFuture<Void, TransactionCommitFailedException> f: futures) {
+                f.checkedGet();
             }
 
+            Optional<NormalizedNode<?, ?>> optional = txChain.newReadOnlyTransaction().read(
+                    LogicalDatastoreType.CONFIGURATION, CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("# cars", nCars, ((Collection<?>)optional.get().getValue()).size());
+
             txChain.close();
 
+            broker.close();
+
             cleanup(dataStore);
         }};
     }
 
     @Test
     public void testCreateChainedTransactionAfterEmptyTxReadied() throws Exception{
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore = setupDistributedDataStore(
                     "testCreateChainedTransactionAfterEmptyTxReadied", "test-1");
 
@@ -714,7 +960,7 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
 
     @Test
     public void testCreateChainedTransactionWhenPreviousNotReady() throws Throwable {
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore = setupDistributedDataStore(
                     "testCreateChainedTransactionWhenPreviousNotReady", "test-1");
 
@@ -734,7 +980,7 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
 
     @Test
     public void testCreateChainedTransactionAfterClose() throws Throwable {
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore = setupDistributedDataStore(
                     "testCreateChainedTransactionAfterClose", "test-1");
 
@@ -748,9 +994,130 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
         }};
     }
 
+    @Test
+    public void testChainWithReadOnlyTxAfterPreviousReady() throws Throwable {
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            DistributedDataStore dataStore = setupDistributedDataStore(
+                    "testChainWithReadOnlyTxAfterPreviousReady", "test-1");
+
+            final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
+
+            // Create a write tx and submit.
+
+            DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
+            writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+            DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
+
+            // Create read-only tx's and issue a read.
+
+            CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readFuture1 =
+                    txChain.newReadOnlyTransaction().read(TestModel.TEST_PATH);
+
+            CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readFuture2 =
+                    txChain.newReadOnlyTransaction().read(TestModel.TEST_PATH);
+
+            // Create another write tx and issue the write.
+
+            DOMStoreWriteTransaction writeTx2 = txChain.newWriteOnlyTransaction();
+            writeTx2.write(TestModel.OUTER_LIST_PATH,
+                    ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
+
+            // Ensure the reads succeed.
+
+            assertEquals("isPresent", true, readFuture1.checkedGet(5, TimeUnit.SECONDS).isPresent());
+            assertEquals("isPresent", true, readFuture2.checkedGet(5, TimeUnit.SECONDS).isPresent());
+
+            // Ensure the writes succeed.
+
+            DOMStoreThreePhaseCommitCohort cohort2 = writeTx2.ready();
+
+            doCommit(cohort1);
+            doCommit(cohort2);
+
+            assertEquals("isPresent", true, txChain.newReadOnlyTransaction().read(TestModel.OUTER_LIST_PATH).
+                    checkedGet(5, TimeUnit.SECONDS).isPresent());
+        }};
+    }
+
+    @Test
+    public void testChainedTransactionFailureWithSingleShard() throws Exception{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            DistributedDataStore dataStore = setupDistributedDataStore(
+                    "testChainedTransactionFailureWithSingleShard", "cars-1");
+
+            ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
+                    ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
+                            LogicalDatastoreType.CONFIGURATION, dataStore).build(), MoreExecutors.directExecutor());
+
+            TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
+            DOMTransactionChain txChain = broker.createTransactionChain(listener);
+
+            DOMDataReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
+
+            ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
+                    new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)).
+                        withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
+
+            rwTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
+
+            try {
+                rwTx.submit().checkedGet(5, TimeUnit.SECONDS);
+                fail("Expected TransactionCommitFailedException");
+            } catch (TransactionCommitFailedException e) {
+                // Expected
+            }
+
+            verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(rwTx), any(Throwable.class));
+
+            txChain.close();
+            broker.close();
+            cleanup(dataStore);
+        }};
+    }
+
+    @Test
+    public void testChainedTransactionFailureWithMultipleShards() throws Exception{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            DistributedDataStore dataStore = setupDistributedDataStore(
+                    "testChainedTransactionFailureWithMultipleShards", "cars-1", "people-1");
+
+            ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
+                    ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
+                            LogicalDatastoreType.CONFIGURATION, dataStore).build(), MoreExecutors.directExecutor());
+
+            TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
+            DOMTransactionChain txChain = broker.createTransactionChain(listener);
+
+            DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
+
+            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();
+
+            // Note that merge will validate the data and fail but put succeeds b/c deep validation is not
+            // done for put for performance reasons.
+            writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
+
+            try {
+                writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
+                fail("Expected TransactionCommitFailedException");
+            } catch (TransactionCommitFailedException e) {
+                // Expected
+            }
+
+            verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
+
+            txChain.close();
+            broker.close();
+            cleanup(dataStore);
+        }};
+    }
+
     @Test
     public void testChangeListenerRegistration() throws Exception{
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore =
                     setupDistributedDataStore("testChangeListenerRegistration", "test-1");
 
@@ -797,128 +1164,51 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
         }};
     }
 
-    class IntegrationTestKit extends ShardTestKit {
-
-        IntegrationTestKit(ActorSystem actorSystem) {
-            super(actorSystem);
-        }
-
-        DistributedDataStore setupDistributedDataStore(String typeName, String... shardNames) {
-            return setupDistributedDataStore(typeName, true, shardNames);
-        }
-
-        DistributedDataStore setupDistributedDataStore(String typeName, boolean waitUntilLeader,
-                String... shardNames) {
-            MockClusterWrapper cluster = new MockClusterWrapper();
-            Configuration config = new ConfigurationImpl("module-shards.conf", "modules.conf");
-            ShardStrategyFactory.setConfiguration(config);
-
-            datastoreContextBuilder.dataStoreType(typeName);
-
-            DatastoreContext datastoreContext = datastoreContextBuilder.build();
-
-            DistributedDataStore dataStore = new DistributedDataStore(getSystem(), cluster,
-                    config, datastoreContext);
-
-            SchemaContext schemaContext = SchemaContextHelper.full();
-            dataStore.onGlobalContextUpdated(schemaContext);
-
-            if(waitUntilLeader) {
-                for(String shardName: shardNames) {
-                    ActorRef shard = null;
-                    for(int i = 0; i < 20 * 5 && shard == null; i++) {
-                        Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
-                        Optional<ActorRef> shardReply = dataStore.getActorContext().findLocalShard(shardName);
-                        if(shardReply.isPresent()) {
-                            shard = shardReply.get();
-                        }
-                    }
-
-                    assertNotNull("Shard was not created", shard);
-
-                    waitUntilLeader(shard);
-                }
-            }
-
-            return dataStore;
-        }
-
-        void testWriteTransaction(DistributedDataStore dataStore, YangInstanceIdentifier nodePath,
-                NormalizedNode<?, ?> nodeToWrite) throws Exception {
+    @Test
+    public void testRestoreFromDatastoreSnapshot() throws Exception{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            String name = "transactionIntegrationTest";
 
-            // 1. Create a write-only Tx
+            ContainerNode carsNode = CarsModel.newCarsNode(CarsModel.newCarsMapNode(
+                    CarsModel.newCarEntry("optima", BigInteger.valueOf(20000L)),
+                    CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000L))));
 
-            DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
-            assertNotNull("newWriteOnlyTransaction returned null", writeTx);
+            ShardDataTree dataTree = new ShardDataTree(SchemaContextHelper.full(), TreeType.OPERATIONAL);
+            AbstractShardTest.writeToStore(dataTree, CarsModel.BASE_PATH, carsNode);
+            NormalizedNode<?, ?> root = AbstractShardTest.readStore(dataTree.getDataTree(),
+                    YangInstanceIdentifier.builder().build());
 
-            // 2. Write some data
+            Snapshot carsSnapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(root),
+                    Collections.<ReplicatedLogEntry>emptyList(), 2, 1, 2, 1, 1, "member-1");
 
-            writeTx.write(nodePath, nodeToWrite);
+            NormalizedNode<?, ?> peopleNode = PeopleModel.create();
+            dataTree = new ShardDataTree(SchemaContextHelper.full(), TreeType.OPERATIONAL);
+            AbstractShardTest.writeToStore(dataTree, PeopleModel.BASE_PATH, peopleNode);
+            root = AbstractShardTest.readStore(dataTree.getDataTree(), YangInstanceIdentifier.builder().build());
 
-            // 3. Ready the Tx for commit
+            Snapshot peopleSnapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(root),
+                    Collections.<ReplicatedLogEntry>emptyList(), 2, 1, 2, 1, 1, "member-1");
 
-            DOMStoreThreePhaseCommitCohort cohort = writeTx.ready();
+            restoreFromSnapshot = new DatastoreSnapshot(name, null, Arrays.asList(
+                    new DatastoreSnapshot.ShardSnapshot("cars",
+                            org.apache.commons.lang3.SerializationUtils.serialize(carsSnapshot)),
+                    new DatastoreSnapshot.ShardSnapshot("people",
+                            org.apache.commons.lang3.SerializationUtils.serialize(peopleSnapshot))));
 
-            // 4. Commit the Tx
-
-            doCommit(cohort);
-
-            // 5. Verify the data in the store
+            DistributedDataStore dataStore = setupDistributedDataStore(name, "module-shards-member1.conf",
+                    true, "cars", "people");
 
             DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
 
-            Optional<NormalizedNode<?, ?>> optional = readTx.read(nodePath).get(5, TimeUnit.SECONDS);
+            Optional<NormalizedNode<?, ?>> optional = readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
             assertEquals("isPresent", true, optional.isPresent());
-            assertEquals("Data node", nodeToWrite, optional.get());
-        }
-
-        void doCommit(final DOMStoreThreePhaseCommitCohort cohort) throws Exception {
-            Boolean canCommit = cohort.canCommit().get(5, TimeUnit.SECONDS);
-            assertEquals("canCommit", true, canCommit);
-            cohort.preCommit().get(5, TimeUnit.SECONDS);
-            cohort.commit().get(5, TimeUnit.SECONDS);
-        }
-
-        void cleanup(DistributedDataStore dataStore) {
-            dataStore.getActorContext().getShardManager().tell(PoisonPill.getInstance(), null);
-        }
-
-        void assertExceptionOnCall(Callable<Void> callable, Class<? extends Exception> expType)
-                throws Exception {
-            try {
-                callable.call();
-                fail("Expected " + expType.getSimpleName());
-            } catch(Exception e) {
-                assertEquals("Exception type", expType, e.getClass());
-            }
-        }
+            assertEquals("Data node", carsNode, optional.get());
 
-        void assertExceptionOnTxChainCreates(final DOMStoreTransactionChain txChain,
-                Class<? extends Exception> expType) throws Exception {
-            assertExceptionOnCall(new Callable<Void>() {
-                @Override
-                public Void call() throws Exception {
-                    txChain.newWriteOnlyTransaction();
-                    return null;
-                }
-            }, expType);
-
-            assertExceptionOnCall(new Callable<Void>() {
-                @Override
-                public Void call() throws Exception {
-                    txChain.newReadWriteTransaction();
-                    return null;
-                }
-            }, expType);
+            optional = readTx.read(PeopleModel.BASE_PATH).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", peopleNode, optional.get());
 
-            assertExceptionOnCall(new Callable<Void>() {
-                @Override
-                public Void call() throws Exception {
-                    txChain.newReadOnlyTransaction();
-                    return null;
-                }
-            }, expType);
-        }
+            cleanup(dataStore);
+        }};
     }
-
 }