Bug 3195: Cleanup on error paths and error handling
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreIntegrationTest.java
index 9f5aded3521b7c72f9b50fff77a988263f1e039d..2319c5be384326a61ef8fb0d5c7519d2e812c498 100644 (file)
@@ -3,33 +3,52 @@ package org.opendaylight.controller.cluster.datastore;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 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.Collection;
 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.InMemoryJournal;
-import org.opendaylight.controller.cluster.datastore.utils.MockClusterWrapper;
 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
+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 +56,40 @@ 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.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 extends AbstractActorTest {
+public class DistributedDataStoreIntegrationTest {
+
+    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 +105,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
 
@@ -148,9 +200,96 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
     }
 
     @Test
-    public void testTransactionWritesWithShardNotInitiallyReady() throws Exception{
-        new IntegrationTestKit(getSystem()) {{
-            String testName = "testTransactionWritesWithShardNotInitiallyReady";
+    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
@@ -163,7 +302,8 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
 
             // Create the write Tx
 
-            final DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
+            final DOMStoreWriteTransaction writeTx = writeOnly ? dataStore.newWriteOnlyTransaction() :
+                    dataStore.newReadWriteTransaction();
             assertNotNull("newReadWriteTransaction returned null", writeTx);
 
             // Do some modification operations and ready the Tx on a separate thread.
@@ -239,8 +379,19 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
     }
 
     @Test
-    public void testTransactionReadsWithShardNotInitiallyReady() throws Exception{
-        new IntegrationTestKit(getSystem()) {{
+    public void testWriteOnlyTransactionWithShardNotInitiallyReady() throws Exception {
+        datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
+        testTransactionWritesWithShardNotInitiallyReady("testWriteOnlyTransactionWithShardNotInitiallyReady", true);
+    }
+
+    @Test
+    public void testReadWriteTransactionWithShardNotInitiallyReady() throws Exception {
+        testTransactionWritesWithShardNotInitiallyReady("testReadWriteTransactionWithShardNotInitiallyReady", false);
+    }
+
+    @Test
+    public void testTransactionReadsWithShardNotInitiallyReady() throws Exception {
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             String testName = "testTransactionReadsWithShardNotInitiallyReady";
             String shardName = "test-1";
 
@@ -313,7 +464,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";
 
@@ -383,7 +534,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";
 
@@ -454,16 +605,16 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
         }};
     }
 
-    @Test(expected=NoShardLeaderException.class)
-    public void testTransactionCommitFailureWithNoShardLeader() throws Throwable{
-        new IntegrationTestKit(getSystem()) {{
+    private void testTransactionCommitFailureWithNoShardLeader(final boolean writeOnly) throws Throwable {
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             String testName = "testTransactionCommitFailureWithNoShardLeader";
-            String shardName = "test-1";
+            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);
 
             // Set the leader election timeout low for the test.
 
@@ -473,7 +624,8 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
 
             // Create the write Tx.
 
-            final DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
+            final DOMStoreWriteTransaction writeTx = writeOnly ? dataStore.newWriteOnlyTransaction() :
+                dataStore.newReadWriteTransaction();
             assertNotNull("newReadWriteTransaction returned null", writeTx);
 
             // Do some modifications and ready the Tx on a separate thread.
@@ -485,8 +637,8 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
                 @Override
                 public void run() {
                     try {
-                        writeTx.write(TestModel.TEST_PATH,
-                                ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                        writeTx.write(TestModel.JUNK_PATH,
+                                ImmutableNodes.containerNode(TestModel.JUNK_QNAME));
 
                         txCohort.set(writeTx.ready());
                     } catch(Exception e) {
@@ -522,10 +674,20 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
         }};
     }
 
+    @Test(expected=NoShardLeaderException.class)
+    public void testWriteOnlyTransactionCommitFailureWithNoShardLeader() throws Throwable {
+        datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
+        testTransactionCommitFailureWithNoShardLeader(true);
+    }
+
+    @Test(expected=NoShardLeaderException.class)
+    public void testReadWriteTransactionCommitFailureWithNoShardLeader() throws Throwable {
+        testTransactionCommitFailureWithNoShardLeader(false);
+    }
+
     @Test
     public void testTransactionAbort() throws Exception{
-        System.setProperty("shard.persistent", "true");
-        new IntegrationTestKit(getSystem()) {{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
             DistributedDataStore dataStore =
                     setupDistributedDataStore("transactionAbortIntegrationTest", "test-1");
 
@@ -548,9 +710,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
 
@@ -635,40 +797,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");
 
@@ -691,7 +936,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");
 
@@ -711,7 +956,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");
 
@@ -725,9 +970,85 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
         }};
     }
 
+    @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");
 
@@ -773,126 +1094,4 @@ public class DistributedDataStoreIntegrationTest extends AbstractActorTest {
             cleanup(dataStore);
         }};
     }
-
-    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);
-
-            DatastoreContext datastoreContext = datastoreContextBuilder.build();
-            DistributedDataStore dataStore = new DistributedDataStore(getSystem(), typeName, 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 {
-
-            // 1. Create a write-only Tx
-
-            DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
-            assertNotNull("newWriteOnlyTransaction returned null", writeTx);
-
-            // 2. Write some data
-
-            writeTx.write(nodePath, nodeToWrite);
-
-            // 3. Ready the Tx for commit
-
-            DOMStoreThreePhaseCommitCohort cohort = writeTx.ready();
-
-            // 4. Commit the Tx
-
-            doCommit(cohort);
-
-            // 5. Verify the data in the store
-
-            DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
-
-            Optional<NormalizedNode<?, ?>> optional = readTx.read(nodePath).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());
-            }
-        }
-
-        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);
-
-            assertExceptionOnCall(new Callable<Void>() {
-                @Override
-                public Void call() throws Exception {
-                    txChain.newReadOnlyTransaction();
-                    return null;
-                }
-            }, expType);
-        }
-    }
-
 }