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 67c3f50a5814f399629369596dcc5e81d3a8d16b..7b78da29234706a56848031169be919a2f3458a7 100644 (file)
@@ -31,7 +31,9 @@ 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.CountDownLatch;
 import java.util.concurrent.ExecutionException;
@@ -43,12 +45,17 @@ 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.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;
@@ -71,6 +78,7 @@ 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.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
 
@@ -489,6 +497,8 @@ public class DistributedDataStoreIntegrationTest {
             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
@@ -559,6 +569,8 @@ public class DistributedDataStoreIntegrationTest {
             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
@@ -616,15 +628,17 @@ public class DistributedDataStoreIntegrationTest {
         }};
     }
 
-    private void testTransactionCommitFailureWithNoShardLeader(final boolean writeOnly) throws Throwable {
+    private void testTransactionCommitFailureWithNoShardLeader(final boolean writeOnly, final String testName) throws Throwable {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
-            String testName = "testTransactionCommitFailureWithNoShardLeader";
             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.
+            // We don't want the shard to become the leader so prevent shard elections.
+            datastoreContextBuilder.customRaftPolicyImplementation(
+                    "org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy");
 
-            datastoreContextBuilder.shardHeartbeatIntervalInMillis(30000);
+            // 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);
 
@@ -632,11 +646,6 @@ public class DistributedDataStoreIntegrationTest {
                     new FindLocalShard(shardName, true));
             assertTrue("Expected LocalShardFound. Actual: " + result, result instanceof LocalShardFound);
 
-            // The ShardManager uses the election timeout for FindPrimary so reset it low so it will timeout quickly.
-
-            datastoreContextBuilder.shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(1);
-            dataStore.onDatastoreContextUpdated(datastoreContextBuilder.build());
-
             // Create the write Tx.
 
             final DOMStoreWriteTransaction writeTx = writeOnly ? dataStore.newWriteOnlyTransaction() :
@@ -692,12 +701,12 @@ public class DistributedDataStoreIntegrationTest {
     @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
@@ -985,6 +994,51 @@ public class DistributedDataStoreIntegrationTest {
         }};
     }
 
+    @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) {{
@@ -1109,4 +1163,52 @@ public class DistributedDataStoreIntegrationTest {
             cleanup(dataStore);
         }};
     }
+
+    @Test
+    public void testRestoreFromDatastoreSnapshot() throws Exception{
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {{
+            String name = "transactionIntegrationTest";
+
+            ContainerNode carsNode = CarsModel.newCarsNode(CarsModel.newCarsMapNode(
+                    CarsModel.newCarEntry("optima", BigInteger.valueOf(20000L)),
+                    CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000L))));
+
+            ShardDataTree dataTree = new ShardDataTree(SchemaContextHelper.full(), TreeType.OPERATIONAL);
+            AbstractShardTest.writeToStore(dataTree, CarsModel.BASE_PATH, carsNode);
+            NormalizedNode<?, ?> root = AbstractShardTest.readStore(dataTree.getDataTree(),
+                    YangInstanceIdentifier.builder().build());
+
+            Snapshot carsSnapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(root),
+                    Collections.<ReplicatedLogEntry>emptyList(), 2, 1, 2, 1, 1, "member-1");
+
+            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());
+
+            Snapshot peopleSnapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(root),
+                    Collections.<ReplicatedLogEntry>emptyList(), 2, 1, 2, 1, 1, "member-1");
+
+            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))));
+
+            DistributedDataStore dataStore = setupDistributedDataStore(name, "module-shards-member1.conf",
+                    true, "cars", "people");
+
+            DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
+
+            Optional<NormalizedNode<?, ?>> optional = readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", carsNode, optional.get());
+
+            optional = readTx.read(PeopleModel.BASE_PATH).get(5, TimeUnit.SECONDS);
+            assertEquals("isPresent", true, optional.isPresent());
+            assertEquals("Data node", peopleNode, optional.get());
+
+            cleanup(dataStore);
+        }};
+    }
 }