X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Ftest%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FShardTest.java;h=921e607eac6740fa628806e28dddcf4291671ddf;hb=4e5083390e8f3076b0c6cd9fc43f656c4a109c1d;hp=a2509fa46c49cb272d0cbd2ce8c26fb36f4bf34f;hpb=466078ab1dc8a8cc2981b161051f6edecd6af85a;p=controller.git diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java index a2509fa46c..921e607eac 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java @@ -7,13 +7,16 @@ */ package org.opendaylight.controller.cluster.datastore; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.endsWith; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; @@ -33,6 +36,7 @@ import akka.util.Timeout; import com.google.common.base.Stopwatch; import com.google.common.base.Throwables; import com.google.common.util.concurrent.Uninterruptibles; +import java.time.Duration; import java.util.Collections; import java.util.HashSet; import java.util.Map; @@ -51,7 +55,6 @@ import org.opendaylight.controller.cluster.access.concepts.MemberName; import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier; import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; -import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats; import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction; import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications; @@ -106,14 +109,14 @@ 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.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.DataTreeCandidate; -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.DataValidationFailedException; import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes; import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder; -import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory; +import org.opendaylight.yangtools.yang.data.tree.api.DataTree; +import org.opendaylight.yangtools.yang.data.tree.api.DataTreeCandidate; +import org.opendaylight.yangtools.yang.data.tree.api.DataTreeConfiguration; +import org.opendaylight.yangtools.yang.data.tree.api.DataTreeModification; +import org.opendaylight.yangtools.yang.data.tree.api.DataValidationFailedException; +import org.opendaylight.yangtools.yang.data.tree.impl.di.InMemoryDataTreeFactory; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; @@ -139,7 +142,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, false), testKit.getRef()); - final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(testKit.duration("3 seconds"), + final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(Duration.ofSeconds(3), RegisterDataTreeNotificationListenerReply.class); final String replyPath = reply.getListenerRegistrationPath().toString(); assertTrue("Incorrect reply path: " + replyPath, @@ -149,6 +152,18 @@ public class ShardTest extends AbstractShardTest { writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); listener.waitForChangeEvents(); + listener.verifyOnInitialDataEvent(); + + final MockDataTreeChangeListener listener2 = new MockDataTreeChangeListener(1); + final ActorRef dclActor2 = actorFactory.createActor(DataTreeChangeListenerActor.props(listener2, + TestModel.TEST_PATH), "testRegisterDataTreeChangeListener-DataTreeChangeListener2"); + + shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor2, false), testKit.getRef()); + + testKit.expectMsgClass(Duration.ofSeconds(3), RegisterDataTreeNotificationListenerReply.class); + + listener2.waitForChangeEvents(); + listener2.verifyNoOnInitialDataEvent(); } @SuppressWarnings("serial") @@ -156,7 +171,7 @@ public class ShardTest extends AbstractShardTest { public void testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception { final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1); final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1); - final Creator creator = new Creator() { + final Creator creator = new Creator<>() { boolean firstElectionTimeout = true; @Override @@ -189,21 +204,20 @@ public class ShardTest extends AbstractShardTest { final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener, path), "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener"); - final TestActorRef shard = actorFactory.createTestActor( - Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), + final TestActorRef shard = actorFactory.createTestActor(Props.create(Shard.class, + new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration"); final ShardTestKit testKit = new ShardTestKit(getSystem()); assertTrue("Got first ElectionTimeout", onFirstElectionTimeout.await(5, TimeUnit.SECONDS)); shard.tell(new RegisterDataTreeChangeListener(path, dclActor, false), testKit.getRef()); - final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), + final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), RegisterDataTreeNotificationListenerReply.class); assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath()); shard.tell(FindLeader.INSTANCE, testKit.getRef()); - final FindLeaderReply findLeadeReply = testKit.expectMsgClass(testKit.duration("5 seconds"), - FindLeaderReply.class); + final FindLeaderReply findLeadeReply = testKit.expectMsgClass(Duration.ofSeconds(5), FindLeaderReply.class); assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent()); onChangeListenerRegistered.countDown(); @@ -224,12 +238,11 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CreateTransaction(nextTransactionId(), TransactionType.READ_ONLY.ordinal(), DataStoreVersions.CURRENT_VERSION).toSerializable(), testKit.getRef()); - final CreateTransactionReply reply = testKit.expectMsgClass(testKit.duration("3 seconds"), + final CreateTransactionReply reply = testKit.expectMsgClass(Duration.ofSeconds(3), CreateTransactionReply.class); final String path = reply.getTransactionPath().toString(); - assertTrue("Unexpected transaction path " + path, path.startsWith(String.format( - "akka://test/user/testCreateTransaction/shard-%s-%s:ShardTransactionTest@0:", + assertThat(path, containsString(String.format("/user/testCreateTransaction/shard-%s-%s:ShardTransactionTest@0:", shardID.getShardName(), shardID.getMemberName().getName()))); } @@ -243,12 +256,12 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CreateTransaction(nextTransactionId(), TransactionType.READ_ONLY.ordinal(), DataStoreVersions.CURRENT_VERSION).toSerializable(), testKit.getRef()); - final CreateTransactionReply reply = testKit.expectMsgClass(testKit.duration("3 seconds"), + final CreateTransactionReply reply = testKit.expectMsgClass(Duration.ofSeconds(3), CreateTransactionReply.class); final String path = reply.getTransactionPath().toString(); - assertTrue("Unexpected transaction path " + path, path.startsWith(String.format( - "akka://test/user/testCreateTransactionOnChain/shard-%s-%s:ShardTransactionTest@0:", + assertThat(path, containsString(String.format( + "/user/testCreateTransactionOnChain/shard-%s-%s:ShardTransactionTest@0:", shardID.getShardName(), shardID.getMemberName().getName()))); } @@ -287,8 +300,8 @@ public class ShardTest extends AbstractShardTest { writeToStore(store, TestModel.TEST_PATH, container); - final YangInstanceIdentifier root = YangInstanceIdentifier.EMPTY; - final NormalizedNode expected = readStore(store, root); + final YangInstanceIdentifier root = YangInstanceIdentifier.empty(); + final NormalizedNode expected = readStore(store, root); final Snapshot snapshot = Snapshot.create(new ShardSnapshotState(new MetadataShardDataTreeSnapshot(expected)), Collections.emptyList(), 1, 2, 3, 4, -1, null, null); @@ -332,7 +345,7 @@ public class ShardTest extends AbstractShardTest { while (sw.elapsed(TimeUnit.SECONDS) <= 5) { Uninterruptibles.sleepUninterruptibly(75, TimeUnit.MILLISECONDS); - final NormalizedNode actual = readStore(shard, TestModel.TEST_PATH); + final NormalizedNode actual = readStore(shard, TestModel.TEST_PATH); if (actual != null) { assertEquals("Applied state", node, actual); return; @@ -361,7 +374,7 @@ public class ShardTest extends AbstractShardTest { // Add some ModificationPayload entries for (int i = 1; i <= nListEntries; i++) { - listEntryKeys.add(Integer.valueOf(i)); + listEntryKeys.add(i); final YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH) .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build(); @@ -377,7 +390,7 @@ public class ShardTest extends AbstractShardTest { InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2, new ApplyJournalEntries(nListEntries)); - testRecovery(listEntryKeys); + testRecovery(listEntryKeys, true); } @Test @@ -387,8 +400,8 @@ public class ShardTest extends AbstractShardTest { final CountDownLatch commitLatch = new CountDownLatch(2); final long timeoutSec = 5; - final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS); - final Timeout timeout = new Timeout(duration); + final Duration duration = Duration.ofSeconds(timeoutSec); + final Timeout timeout = Timeout.create(duration); final TestActorRef shard = actorFactory.createTestActor( newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), @@ -467,7 +480,9 @@ public class ShardTest extends AbstractShardTest { ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), testKit.getRef()); final ReadyTransactionReply readyReply = ReadyTransactionReply .fromSerializable(testKit.expectMsgClass(duration, ReadyTransactionReply.class)); - assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath()); + + String pathSuffix = shard.path().toString().replaceFirst("akka://test", ""); + assertThat(readyReply.getCohortPath(), endsWith(pathSuffix)); // Send the CanCommitTransaction message for the first Tx. shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), testKit.getRef()); @@ -539,7 +554,7 @@ public class ShardTest extends AbstractShardTest { verifyOuterListEntry(shard, 1); - verifyLastApplied(shard, 5); + verifyLastApplied(shard, 3); } @Test @@ -552,7 +567,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); final TransactionIdentifier transactionID = nextTransactionId(); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); // Send a BatchedModifications to start a transaction. @@ -601,7 +616,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); final TransactionIdentifier transactionID = nextTransactionId(); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); // Send a BatchedModifications to start a transaction. @@ -645,7 +660,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(batched, testKit.getRef()); - final Failure failure = testKit.expectMsgClass(testKit.duration("5 seconds"), Failure.class); + final Failure failure = testKit.expectMsgClass(Duration.ofSeconds(5), Failure.class); if (failure != null) { Throwables.propagateIfPossible(failure.cause(), Exception.class); @@ -675,7 +690,7 @@ public class ShardTest extends AbstractShardTest { BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION); batched.addModification(new MergeModification(TestModel.TEST_PATH, invalidData)); shard.tell(batched, testKit.getRef()); - Failure failure = testKit.expectMsgClass(testKit.duration("5 seconds"), akka.actor.Status.Failure.class); + Failure failure = testKit.expectMsgClass(Duration.ofSeconds(5), akka.actor.Status.Failure.class); final Throwable cause = failure.cause(); @@ -685,7 +700,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(batched, testKit.getRef()); - failure = testKit.expectMsgClass(testKit.duration("5 seconds"), akka.actor.Status.Failure.class); + failure = testKit.expectMsgClass(Duration.ofSeconds(5), akka.actor.Status.Failure.class); assertEquals("Failure cause", cause, failure.cause()); } @@ -702,7 +717,7 @@ public class ShardTest extends AbstractShardTest { final TransactionIdentifier transactionID1 = new TransactionIdentifier(historyId, 0); final TransactionIdentifier transactionID2 = new TransactionIdentifier(historyId, 1); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); // Send a BatchedModifications to start a chained write // transaction and ready it. @@ -717,12 +732,12 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CreateTransaction(transactionID2, TransactionType.READ_ONLY.ordinal(), DataStoreVersions.CURRENT_VERSION).toSerializable(), testKit.getRef()); - final CreateTransactionReply createReply = testKit.expectMsgClass(testKit.duration("3 seconds"), + final CreateTransactionReply createReply = testKit.expectMsgClass(Duration.ofSeconds(3), CreateTransactionReply.class); getSystem().actorSelection(createReply.getTransactionPath()) .tell(new ReadData(path, DataStoreVersions.CURRENT_VERSION), testKit.getRef()); - final ReadDataReply readReply = testKit.expectMsgClass(testKit.duration("3 seconds"), ReadDataReply.class); + final ReadDataReply readReply = testKit.expectMsgClass(Duration.ofSeconds(3), ReadDataReply.class); assertEquals("Read node", containerNode, readReply.getNormalizedNode()); // Commit the write transaction. @@ -737,7 +752,7 @@ public class ShardTest extends AbstractShardTest { // Verify data in the data store. - final NormalizedNode actualNode = readStore(shard, path); + final NormalizedNode actualNode = readStore(shard, path); assertEquals("Stored node", containerNode, actualNode); } @@ -745,7 +760,7 @@ public class ShardTest extends AbstractShardTest { public void testOnBatchedModificationsWhenNotLeader() { final AtomicBoolean overrideLeaderCalls = new AtomicBoolean(); final ShardTestKit testKit = new ShardTestKit(getSystem()); - final Creator creator = new Creator() { + final Creator creator = new Creator<>() { private static final long serialVersionUID = 1L; @Override @@ -765,8 +780,8 @@ public class ShardTest extends AbstractShardTest { } }; - final TestActorRef shard = actorFactory.createTestActor(Props - .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), + final TestActorRef shard = actorFactory.createTestActor(Props.create(Shard.class, + new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "testOnBatchedModificationsWhenNotLeader"); ShardTestKit.waitUntilLeader(shard); @@ -827,7 +842,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); final TransactionIdentifier transactionID = nextTransactionId(); - final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); if (readWrite) { shard.tell(prepareForwardedReadyTransaction(shard, transactionID, TestModel.TEST_PATH, containerNode, true), testKit.getRef()); @@ -836,9 +851,9 @@ public class ShardTest extends AbstractShardTest { testKit.getRef()); } - testKit.expectMsgClass(testKit.duration("5 seconds"), CommitTransactionReply.class); + testKit.expectMsgClass(Duration.ofSeconds(5), CommitTransactionReply.class); - final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); + final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode); } @@ -871,7 +886,7 @@ public class ShardTest extends AbstractShardTest { testKit.expectMsgClass(CommitTransactionReply.class); - final NormalizedNode actualNode = readStore(shard, TestModel.OUTER_LIST_PATH); + final NormalizedNode actualNode = readStore(shard, TestModel.OUTER_LIST_PATH); assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode); } @@ -916,7 +931,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CommitTransaction(txId, CURRENT_VERSION).toSerializable(), testKit.getRef()); testKit.expectMsgClass(CommitTransactionReply.class); - final NormalizedNode actualNode = readStore(shard, TestModel.OUTER_LIST_PATH); + final NormalizedNode actualNode = readStore(shard, TestModel.OUTER_LIST_PATH); assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode); } @@ -932,10 +947,10 @@ public class ShardTest extends AbstractShardTest { // Setup a simulated transactions with a mock cohort. - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); final TransactionIdentifier transactionID = nextTransactionId(); - final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); + final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME); shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH, containerNode, false), testKit.getRef()); testKit.expectMsgClass(duration, ReadyTransactionReply.class); @@ -952,7 +967,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), testKit.getRef()); testKit.expectMsgClass(duration, CommitTransactionReply.class); - final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); + final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH); assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode); } @@ -975,7 +990,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); final TransactionIdentifier transactionID = nextTransactionId(); if (readWrite) { @@ -1014,8 +1029,7 @@ public class ShardTest extends AbstractShardTest { // Committed transaction count should increase as usual assertEquals(1, shardStats.getCommittedTransactionsCount()); - // Commit index should advance as we do not have an empty - // modification + // Commit index should advance 1 to account for disabling metadata assertEquals(1, shardStats.getCommitIndex()); } @@ -1029,8 +1043,8 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); - final Timeout timeout = new Timeout(duration); + final Duration duration = Duration.ofSeconds(5); + final Timeout timeout = Timeout.create(duration); // Setup 2 simulated transactions with mock cohorts. The first // one fails in the @@ -1073,7 +1087,7 @@ public class ShardTest extends AbstractShardTest { // Wait for the 2nd Tx to complete the canCommit phase. final CountDownLatch latch = new CountDownLatch(1); - canCommitFuture.onComplete(new OnComplete() { + canCommitFuture.onComplete(new OnComplete<>() { @Override public void onComplete(final Throwable failure, final Object resp) { latch.countDown(); @@ -1103,8 +1117,8 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); - final Timeout timeout = new Timeout(duration); + final Duration duration = Duration.ofSeconds(5); + final Timeout timeout = Timeout.create(duration); doThrow(new RuntimeException("mock preCommit failure")).when(dataTree) .prepare(any(DataTreeModification.class)); @@ -1143,7 +1157,7 @@ public class ShardTest extends AbstractShardTest { // Wait for the 2nd Tx to complete the canCommit phase. final CountDownLatch latch = new CountDownLatch(1); - canCommitFuture.onComplete(new OnComplete() { + canCommitFuture.onComplete(new OnComplete<>() { @Override public void onComplete(final Throwable failure, final Object resp) { latch.countDown(); @@ -1168,10 +1182,10 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); final TransactionIdentifier transactionID1 = nextTransactionId(); - doThrow(new DataValidationFailedException(YangInstanceIdentifier.EMPTY, "mock canCommit failure")) + doThrow(new DataValidationFailedException(YangInstanceIdentifier.empty(), "mock canCommit failure")) .doNothing().when(dataTree).validate(any(DataTreeModification.class)); shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, @@ -1212,10 +1226,10 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - doThrow(new DataValidationFailedException(YangInstanceIdentifier.EMPTY, "mock canCommit failure")) + doThrow(new DataValidationFailedException(YangInstanceIdentifier.empty(), "mock canCommit failure")) .doNothing().when(dataTree).validate(any(DataTreeModification.class)); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); final TransactionIdentifier transactionID1 = nextTransactionId(); @@ -1261,13 +1275,13 @@ public class ShardTest extends AbstractShardTest { } }; - final TestActorRef shard = actorFactory.createTestActor(Props - .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), + final TestActorRef shard = actorFactory.createTestActor(Props.create(Shard.class, + new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "testAbortWithCommitPending"); ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); final TransactionIdentifier transactionID = nextTransactionId(); @@ -1281,7 +1295,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), testKit.getRef()); testKit.expectMsgClass(duration, CommitTransactionReply.class); - final NormalizedNode node = readStore(shard, TestModel.TEST_PATH); + final NormalizedNode node = readStore(shard, TestModel.TEST_PATH); // Since we're simulating an abort occurring during replication // and before finish commit, @@ -1301,7 +1315,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); writeToStore(shard, TestModel.OUTER_LIST_PATH, @@ -1349,7 +1363,7 @@ public class ShardTest extends AbstractShardTest { shard.tell(new CommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), testKit.getRef()); testKit.expectMsgClass(duration, CommitTransactionReply.class); - final NormalizedNode node = readStore(shard, listNodePath); + final NormalizedNode node = readStore(shard, listNodePath); assertNotNull(listNodePath + " not found", node); } @@ -1430,7 +1444,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); final TransactionIdentifier transactionID1 = nextTransactionId(); shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH, @@ -1466,7 +1480,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); final ShardDataTree dataStore = shard.underlyingActor().getDataStore(); @@ -1509,7 +1523,7 @@ public class ShardTest extends AbstractShardTest { testKit.expectMsgClass(duration, CommitTransactionReply.class); - final NormalizedNode node = readStore(shard, TestModel.TEST2_PATH); + final NormalizedNode node = readStore(shard, TestModel.TEST2_PATH); assertNotNull(TestModel.TEST2_PATH + " not found", node); } @@ -1521,7 +1535,7 @@ public class ShardTest extends AbstractShardTest { "testCanCommitBeforeReadyFailure"); shard.tell(new CanCommitTransaction(nextTransactionId(), CURRENT_VERSION).toSerializable(), testKit.getRef()); - testKit.expectMsgClass(testKit.duration("5 seconds"), akka.actor.Status.Failure.class); + testKit.expectMsgClass(Duration.ofSeconds(5), akka.actor.Status.Failure.class); } @Test @@ -1532,8 +1546,8 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); - final Timeout timeout = new Timeout(duration); + final Duration duration = Duration.ofSeconds(5); + final Timeout timeout = Timeout.create(duration); // Ready 2 transactions - the first one will be aborted. @@ -1570,7 +1584,8 @@ public class ShardTest extends AbstractShardTest { // Wait for the 2nd Tx to complete the canCommit phase. - canCommitReply = (CanCommitTransactionReply) Await.result(canCommitFuture, duration); + canCommitReply = (CanCommitTransactionReply) Await.result(canCommitFuture, + FiniteDuration.create(5, TimeUnit.SECONDS)); assertTrue("Can commit", canCommitReply.getCanCommit()); } @@ -1583,7 +1598,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); // Ready a tx. @@ -1624,7 +1639,7 @@ public class ShardTest extends AbstractShardTest { ShardTestKit.waitUntilLeader(shard); - final FiniteDuration duration = testKit.duration("5 seconds"); + final Duration duration = Duration.ofSeconds(5); // Ready 3 tx's. @@ -1693,9 +1708,9 @@ public class ShardTest extends AbstractShardTest { dataStoreContextBuilder.persistent(persistent); - class TestShard extends Shard { + final class TestShard extends Shard { - protected TestShard(final AbstractBuilder builder) { + TestShard(final AbstractBuilder builder) { super(builder); setPersistence(new TestPersistentDataProvider(super.persistence())); } @@ -1716,18 +1731,15 @@ public class ShardTest extends AbstractShardTest { } } - final ShardTestKit testKit = new ShardTestKit(getSystem()); - final Creator creator = () -> new TestShard(newShardBuilder()); - final TestActorRef shard = actorFactory.createTestActor(Props - .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), - shardActorName); + final TestActorRef shard = actorFactory.createTestActor(Props.create(Shard.class, + new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), shardActorName); ShardTestKit.waitUntilLeader(shard); writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); - final NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.EMPTY); + final NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.empty()); // Trigger creation of a snapshot by ensuring final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext(); @@ -1739,7 +1751,7 @@ public class ShardTest extends AbstractShardTest { } private static void awaitAndValidateSnapshot(final AtomicReference latch, - final AtomicReference savedSnapshot, final NormalizedNode expectedRoot) + final AtomicReference savedSnapshot, final NormalizedNode expectedRoot) throws InterruptedException { assertTrue("Snapshot saved", latch.get().await(5, TimeUnit.SECONDS)); @@ -1751,8 +1763,8 @@ public class ShardTest extends AbstractShardTest { savedSnapshot.set(null); } - private static void verifySnapshot(final Snapshot snapshot, final NormalizedNode expectedRoot) { - final NormalizedNode actual = ((ShardSnapshotState)snapshot.getState()).getSnapshot().getRootNode().get(); + private static void verifySnapshot(final Snapshot snapshot, final NormalizedNode expectedRoot) { + final NormalizedNode actual = ((ShardSnapshotState)snapshot.getState()).getSnapshot().getRootNode().get(); assertEquals("Root node", expectedRoot, actual); } @@ -1770,16 +1782,16 @@ public class ShardTest extends AbstractShardTest { commitTransaction(store, putTransaction); - final NormalizedNode expected = readStore(store, YangInstanceIdentifier.EMPTY); + final NormalizedNode expected = readStore(store, YangInstanceIdentifier.empty()); final DataTreeModification writeTransaction = store.takeSnapshot().newModification(); - writeTransaction.delete(YangInstanceIdentifier.EMPTY); - writeTransaction.write(YangInstanceIdentifier.EMPTY, expected); + writeTransaction.delete(YangInstanceIdentifier.empty()); + writeTransaction.write(YangInstanceIdentifier.empty(), expected); commitTransaction(store, writeTransaction); - final NormalizedNode actual = readStore(store, YangInstanceIdentifier.EMPTY); + final NormalizedNode actual = readStore(store, YangInstanceIdentifier.empty()); assertEquals(expected, actual); } @@ -1895,7 +1907,7 @@ public class ShardTest extends AbstractShardTest { testKit.waitUntilNoLeader(shard); shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), testKit.getRef()); - final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), + final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), RegisterDataTreeNotificationListenerReply.class); assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath()); @@ -1925,7 +1937,7 @@ public class ShardTest extends AbstractShardTest { testKit.waitUntilNoLeader(shard); shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), testKit.getRef()); - final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), + final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), RegisterDataTreeNotificationListenerReply.class); assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath()); @@ -1974,7 +1986,7 @@ public class ShardTest extends AbstractShardTest { actorFactory.generateActorId(testName + "-DataTreeChangeListener")); followerShard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), testKit.getRef()); - final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), + final RegisterDataTreeNotificationListenerReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), RegisterDataTreeNotificationListenerReply.class); assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());