X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Ftest%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FDistributedDataStoreRemotingIntegrationTest.java;h=b78d017341da3cc860c725ec4a95be9ad899b1d9;hp=63d785c984659a39c41664637e0fef5860259143;hb=20a32e6459fd1e27e7669bf1ebc7742b96787b94;hpb=0f7dbbe9b8b3d7ddb22de34ac04650b7ba7fc440 diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreRemotingIntegrationTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreRemotingIntegrationTest.java index 63d785c984..b78d017341 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreRemotingIntegrationTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreRemotingIntegrationTest.java @@ -53,6 +53,7 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; import org.mockito.Mockito; import org.mockito.stubbing.Answer; +import org.opendaylight.controller.cluster.access.client.RequestTimeoutException; import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier; import org.opendaylight.controller.cluster.databroker.ClientBackedDataStore; import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker; @@ -68,6 +69,7 @@ import org.opendaylight.controller.cluster.datastore.modification.MergeModificat import org.opendaylight.controller.cluster.datastore.modification.WriteModification; import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot; import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState; +import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow; import org.opendaylight.controller.cluster.raft.client.messages.Shutdown; import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries; import org.opendaylight.controller.cluster.raft.persisted.Snapshot; @@ -95,9 +97,8 @@ 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; @@ -117,7 +118,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { @Parameters(name = "{0}") public static Collection data() { return Arrays.asList(new Object[][] { - { DistributedDataStore.class, 7}, { ClientBackedDataStore.class, 60 } + { DistributedDataStore.class, 7}, { ClientBackedDataStore.class, 12 } }); } @@ -137,6 +138,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { private static final String MODULE_SHARDS_CARS_ONLY_1_2 = "module-shards-cars-member-1-and-2.conf"; private static final String MODULE_SHARDS_CARS_PEOPLE_1_2 = "module-shards-member1-and-2.conf"; private static final String MODULE_SHARDS_CARS_PEOPLE_1_2_3 = "module-shards-member1-and-2-and-3.conf"; + private static final String MODULE_SHARDS_CARS_1_2_3 = "module-shards-cars-member-1-and-2-and-3.conf"; private ActorSystem leaderSystem; private ActorSystem followerSystem; @@ -574,6 +576,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { @Test public void testSingleShardTransactionsWithLeaderChanges() throws Exception { + followerDatastoreContextBuilder.backendAlivenessTimerIntervalInSeconds(2); final String testName = "testSingleShardTransactionsWithLeaderChanges"; initDatastoresWithCars(testName); @@ -638,8 +641,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. @@ -782,10 +785,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. @@ -845,7 +848,6 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()) .shardElectionTimeoutFactor(10)); - Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS); leaderTestKit.waitUntilNoLeader(leaderDistributedDataStore.getActorContext(), "cars"); // Submit all tx's - the messages should get queued for retry. @@ -860,6 +862,10 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder .customRaftPolicyImplementation(null).shardElectionTimeoutFactor(1)); + IntegrationTestKit.findLocalShard(followerDistributedDataStore.getActorContext(), "cars") + .tell(TimeoutNow.INSTANCE, ActorRef.noSender()); + IntegrationTestKit.findLocalShard(followerDistributedDataStore.getActorContext(), "people") + .tell(TimeoutNow.INSTANCE, ActorRef.noSender()); followerTestKit.doCommit(writeTx1CanCommit, writeTx1Cohort); followerTestKit.doCommit(writeTx2CanCommit, writeTx2Cohort); @@ -994,6 +1000,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { @Test public void testTransactionWithShardLeaderNotResponding() throws Exception { + followerDatastoreContextBuilder.frontendRequestTimeoutInSeconds(2); followerDatastoreContextBuilder.shardElectionTimeoutFactor(50); initDatastoresWithCars("testTransactionWithShardLeaderNotResponding"); @@ -1018,13 +1025,18 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { fail("Exception expected"); } catch (final ExecutionException e) { final String msg = "Unexpected exception: " + Throwables.getStackTraceAsString(e.getCause()); - assertTrue(msg, Throwables.getRootCause(e) instanceof NoShardLeaderException - || e.getCause() instanceof ShardLeaderNotRespondingException); + if (DistributedDataStore.class.equals(testParameter)) { + assertTrue(msg, Throwables.getRootCause(e) instanceof NoShardLeaderException + || e.getCause() instanceof ShardLeaderNotRespondingException); + } else { + assertTrue(msg, Throwables.getRootCause(e) instanceof RequestTimeoutException); + } } } @Test public void testTransactionWithCreateTxFailureDueToNoLeader() throws Exception { + followerDatastoreContextBuilder.frontendRequestTimeoutInSeconds(2); initDatastoresWithCars("testTransactionWithCreateTxFailureDueToNoLeader"); // Do an initial read to get the primary shard info cached. @@ -1051,25 +1063,29 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { followerTestKit.doCommit(rwTx.ready()); fail("Exception expected"); } catch (final ExecutionException e) { - final String msg = "Expected instance of NoShardLeaderException, actual: \n" - + Throwables.getStackTraceAsString(e.getCause()); - assertTrue(msg, Throwables.getRootCause(e) instanceof NoShardLeaderException); + final String msg = "Unexpected exception: " + Throwables.getStackTraceAsString(e.getCause()); + if (DistributedDataStore.class.equals(testParameter)) { + assertTrue(msg, Throwables.getRootCause(e) instanceof NoShardLeaderException); + } else { + assertTrue(msg, Throwables.getRootCause(e) instanceof RequestTimeoutException); + } } } @Test public void testTransactionRetryWithInitialAskTimeoutExOnCreateTx() throws Exception { + followerDatastoreContextBuilder.backendAlivenessTimerIntervalInSeconds(2); String testName = "testTransactionRetryWithInitialAskTimeoutExOnCreateTx"; - initDatastores(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, CARS); + initDatastores(testName, MODULE_SHARDS_CARS_1_2_3, CARS); final DatastoreContext.Builder follower2DatastoreContextBuilder = DatastoreContext.newBuilder() - .shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5); + .shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(10); final IntegrationTestKit follower2TestKit = new IntegrationTestKit( follower2System, follower2DatastoreContextBuilder, commitTimeout); try (AbstractDataStore ds = follower2TestKit.setupAbstractDataStore( - testParameter, testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, false, CARS)) { + testParameter, testName, MODULE_SHARDS_CARS_1_2_3, false, CARS)) { followerTestKit.waitForMembersUp("member-1", "member-3"); follower2TestKit.waitForMembersUp("member-1", "member-2"); @@ -1086,7 +1102,7 @@ public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest { Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS); sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder - .operationTimeoutInMillis(500).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null)); + .operationTimeoutInMillis(500).shardElectionTimeoutFactor(5).customRaftPolicyImplementation(null)); final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction(); @@ -1105,8 +1121,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)))); @@ -1135,6 +1151,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());