Convert mdsal submit() calls to commit()
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreRemotingIntegrationTest.java
index a2a5e63a2b994e64f18cd14f918d4f3e1d6fff31..aaab6b19e18d21b0d960daaf4db3250eaa76afac 100644 (file)
@@ -24,7 +24,7 @@ import akka.actor.AddressFromURIString;
 import akka.cluster.Cluster;
 import akka.dispatch.Futures;
 import akka.pattern.Patterns;
-import akka.testkit.JavaTestKit;
+import akka.testkit.javadsl.TestKit;
 import com.google.common.base.Optional;
 import com.google.common.base.Stopwatch;
 import com.google.common.base.Supplier;
@@ -80,26 +80,24 @@ 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.LogicalDatastoreType;
-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.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;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
+import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
+import org.opendaylight.mdsal.common.api.TransactionChainListener;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
+import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
+import org.opendaylight.mdsal.dom.spi.store.DOMStore;
+import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadTransaction;
+import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadWriteTransaction;
+import org.opendaylight.mdsal.dom.spi.store.DOMStoreThreePhaseCommitCohort;
+import org.opendaylight.mdsal.dom.spi.store.DOMStoreTransactionChain;
+import org.opendaylight.mdsal.dom.spi.store.DOMStoreWriteTransaction;
 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.DataTree;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeConfiguration;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
-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.api.CollectionNodeBuilder;
 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
@@ -183,9 +181,9 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
             leaderDistributedDataStore.close();
         }
 
-        JavaTestKit.shutdownActorSystem(leaderSystem);
-        JavaTestKit.shutdownActorSystem(followerSystem);
-        JavaTestKit.shutdownActorSystem(follower2System);
+        TestKit.shutdownActorSystem(leaderSystem);
+        TestKit.shutdownActorSystem(followerSystem);
+        TestKit.shutdownActorSystem(follower2System);
 
         InMemoryJournal.clear();
         InMemorySnapshotStore.clear();
@@ -317,8 +315,8 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
             Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
         }
 
-        JavaTestKit.shutdownActorSystem(leaderSystem, null, Boolean.TRUE);
-        JavaTestKit.shutdownActorSystem(followerSystem, null, Boolean.TRUE);
+        TestKit.shutdownActorSystem(leaderSystem, Boolean.TRUE);
+        TestKit.shutdownActorSystem(followerSystem, Boolean.TRUE);
 
         final ActorSystem newSystem = newActorSystem("reinstated-member2", "Member2");
 
@@ -517,7 +515,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
         final TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
         final DOMTransactionChain txChain = broker.createTransactionChain(listener);
 
-        final DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
+        final DOMDataTreeWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
 
         final ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
@@ -526,9 +524,9 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
 
         try {
-            writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
+            writeTx.commit().get(5, TimeUnit.SECONDS);
             fail("Expected TransactionCommitFailedException");
-        } catch (final TransactionCommitFailedException e) {
+        } catch (final ExecutionException e) {
             // Expected
         }
 
@@ -550,7 +548,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
         final TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
         final DOMTransactionChain txChain = broker.createTransactionChain(listener);
 
-        final DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
+        final DOMDataTreeWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
 
         writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
 
@@ -563,9 +561,9 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
 
         try {
-            writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
+            writeTx.commit().get(5, TimeUnit.SECONDS);
             fail("Expected TransactionCommitFailedException");
-        } catch (final TransactionCommitFailedException e) {
+        } catch (final ExecutionException e) {
             // Expected
         }
 
@@ -600,7 +598,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
                 .shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
 
-        JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
+        TestKit.shutdownActorSystem(leaderSystem, true);
         Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
 
         followerTestKit.waitUntilNoLeader(followerDistributedDataStore.getActorContext(), CARS);
@@ -642,8 +640,8 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
                 .findLocalShard("cars");
         assertEquals("Cars follower shard found", true, carsFollowerShard.isPresent());
 
-        final TipProducingDataTree dataTree = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
-        dataTree.setSchemaContext(SchemaContextHelper.full());
+        final DataTree dataTree = new InMemoryDataTreeFactory().create(
+            DataTreeConfiguration.DEFAULT_OPERATIONAL, SchemaContextHelper.full());
 
         // Send a tx with immediate commit.
 
@@ -786,10 +784,10 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
         // Wait for the commit to be replicated to the follower.
 
         MemberNode.verifyRaftState(followerDistributedDataStore, "cars",
-            raftState -> assertEquals("getLastApplied", 0, raftState.getLastApplied()));
+            raftState -> assertEquals("getLastApplied", 1, raftState.getLastApplied()));
 
         MemberNode.verifyRaftState(followerDistributedDataStore, "people",
-            raftState -> assertEquals("getLastApplied", 0, raftState.getLastApplied()));
+            raftState -> assertEquals("getLastApplied", 1, raftState.getLastApplied()));
 
         // Prepare, ready and canCommit a WO tx that writes to 2 shards. This will become the current tx in
         // the leader shard.
@@ -1012,7 +1010,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
 
         // Shutdown the leader and try to create a new tx.
 
-        JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
+        TestKit.shutdownActorSystem(leaderSystem, true);
 
         followerDatastoreContextBuilder.operationTimeoutInMillis(50).shardElectionTimeoutFactor(1);
         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder);
@@ -1047,7 +1045,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
 
         // Shutdown the leader and try to create a new tx.
 
-        JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
+        TestKit.shutdownActorSystem(leaderSystem, true);
 
         Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
 
@@ -1098,7 +1096,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
 
             // Shutdown the leader and try to create a new tx.
 
-            JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
+            TestKit.shutdownActorSystem(leaderSystem, true);
 
             Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
 
@@ -1122,8 +1120,8 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
         // Setup a saved snapshot on the leader. The follower will startup with no data and the leader should
         // install a snapshot to sync the follower.
 
-        TipProducingDataTree tree = InMemoryDataTreeFactory.getInstance().create(TreeType.CONFIGURATION);
-        tree.setSchemaContext(SchemaContextHelper.full());
+        DataTree tree = new InMemoryDataTreeFactory().create(DataTreeConfiguration.DEFAULT_CONFIGURATION,
+            SchemaContextHelper.full());
 
         final ContainerNode carsNode = CarsModel.newCarsNode(
                 CarsModel.newCarsMapNode(CarsModel.newCarEntry("optima", BigInteger.valueOf(20000))));
@@ -1152,6 +1150,23 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
                 initialSnapshot, snapshotRoot);
     }
 
+    @Test
+    public void testReadWriteMessageSlicing() throws Exception {
+        // The slicing is only implemented for tell-based protocol
+        Assume.assumeTrue(testParameter.equals(ClientBackedDataStore.class));
+
+        leaderDatastoreContextBuilder.maximumMessageSliceSize(100);
+        followerDatastoreContextBuilder.maximumMessageSliceSize(100);
+        initDatastoresWithCars("testLargeReadReplySlicing");
+
+        final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
+
+        final NormalizedNode<?, ?> carsNode = CarsModel.create();
+        rwTx.write(CarsModel.BASE_PATH, carsNode);
+
+        verifyNode(rwTx, CarsModel.BASE_PATH, carsNode);
+    }
+
     private static void verifySnapshot(final Snapshot actual, final Snapshot expected,
                                        final NormalizedNode<?, ?> expRoot) {
         assertEquals("Snapshot getLastAppliedTerm", expected.getLastAppliedTerm(), actual.getLastAppliedTerm());