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%2FAbstractShardTest.java;h=cc2e29e211ef590688fabcd876dc6bdc3be31974;hp=1100f3a7fa2c0fd584dc57618a6cd54339aa4b60;hb=c796596b5c46b5203c30b143e6282662e66c5642;hpb=e20fff4d018e95cefd1934d2be31e5cd692fe7fa diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractShardTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractShardTest.java index 1100f3a7fa..cc2e29e211 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractShardTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractShardTest.java @@ -11,16 +11,23 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION; import akka.actor.ActorRef; import akka.actor.PoisonPill; import akka.actor.Props; import akka.dispatch.Dispatchers; import akka.japi.Creator; +import akka.pattern.Patterns; import akka.testkit.TestActorRef; +import akka.util.Timeout; import com.google.common.base.Function; import com.google.common.base.Optional; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.Uninterruptibles; import java.util.Collections; @@ -28,19 +35,29 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.opendaylight.controller.cluster.access.concepts.MemberName; import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; +import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications; +import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction; import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification; import org.opendaylight.controller.cluster.datastore.modification.WriteModification; +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.TestActorFactory; import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal; import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore; +import org.opendaylight.controller.md.cluster.datastore.model.CarsModel; import org.opendaylight.controller.md.cluster.datastore.model.TestModel; +import org.opendaylight.yangtools.concepts.Identifier; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument; import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild; @@ -48,11 +65,18 @@ import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode; 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.DataTreeCandidateNode; import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip; import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification; -import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeSnapshot; import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException; +import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType; +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.tree.InMemoryDataTreeFactory; import org.opendaylight.yangtools.yang.model.api.SchemaContext; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Duration; /** * Abstract base for shard unit tests. @@ -64,13 +88,15 @@ public abstract class AbstractShardTest extends AbstractActorTest{ private static final AtomicInteger NEXT_SHARD_NUM = new AtomicInteger(); - protected final ShardIdentifier shardID = ShardIdentifier.builder().memberName("member-1") - .shardName("inventory").type("config" + NEXT_SHARD_NUM.getAndIncrement()).build(); + protected final ShardIdentifier shardID = ShardIdentifier.create("inventory", MemberName.forName("member-1"), + "config" + NEXT_SHARD_NUM.getAndIncrement()); protected final Builder dataStoreContextBuilder = DatastoreContext.newBuilder(). shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000). shardHeartbeatIntervalInMillis(100); + protected final TestActorFactory actorFactory = new TestActorFactory(getSystem()); + @Before public void setUp() { InMemorySnapshotStore.clear(); @@ -81,6 +107,7 @@ public abstract class AbstractShardTest extends AbstractActorTest{ public void tearDown() { InMemorySnapshotStore.clear(); InMemoryJournal.clear(); + actorFactory.close(); } protected DatastoreContext newDatastoreContext() { @@ -88,23 +115,25 @@ public abstract class AbstractShardTest extends AbstractActorTest{ } protected Props newShardProps() { - return Shard.props(shardID, Collections.emptyMap(), - newDatastoreContext(), SCHEMA_CONTEXT); + return newShardBuilder().props(); + } + + protected Shard.Builder newShardBuilder() { + return Shard.builder().id(shardID).datastoreContext(newDatastoreContext()).schemaContext(SCHEMA_CONTEXT); } - protected void testRecovery(Set listEntryKeys) throws Exception { + protected void testRecovery(final Set listEntryKeys) throws Exception { // Create the actor and wait for recovery complete. - int nListEntries = listEntryKeys.size(); + final int nListEntries = listEntryKeys.size(); final CountDownLatch recoveryComplete = new CountDownLatch(1); @SuppressWarnings("serial") - Creator creator = new Creator() { + final Creator creator = new Creator() { @Override public Shard create() throws Exception { - return new Shard(shardID, Collections.emptyMap(), - newDatastoreContext(), SCHEMA_CONTEXT) { + return new Shard(newShardBuilder()) { @Override protected void onRecoveryComplete() { try { @@ -117,25 +146,25 @@ public abstract class AbstractShardTest extends AbstractActorTest{ } }; - TestActorRef shard = TestActorRef.create(getSystem(), + final TestActorRef shard = TestActorRef.create(getSystem(), Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), "testRecovery"); assertEquals("Recovery complete", true, recoveryComplete.await(5, TimeUnit.SECONDS)); // Verify data in the data store. - NormalizedNode outerList = readStore(shard, TestModel.OUTER_LIST_PATH); + final NormalizedNode outerList = readStore(shard, TestModel.OUTER_LIST_PATH); assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList); assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable", outerList.getValue() instanceof Iterable); - for(Object entry: (Iterable) outerList.getValue()) { + for(final Object entry: (Iterable) outerList.getValue()) { assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode", entry instanceof MapEntryNode); - MapEntryNode mapEntry = (MapEntryNode)entry; - Optional> idLeaf = + final MapEntryNode mapEntry = (MapEntryNode)entry; + final Optional> idLeaf = mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME)); assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent()); - Object value = idLeaf.get().getValue(); + final Object value = idLeaf.get().getValue(); assertTrue("Unexpected value for leaf "+ TestModel.ID_QNAME.getLocalName() + ": " + value, listEntryKeys.remove(value)); } @@ -155,7 +184,7 @@ public abstract class AbstractShardTest extends AbstractActorTest{ shard.tell(PoisonPill.getInstance(), ActorRef.noSender()); } - protected void verifyLastApplied(TestActorRef shard, long expectedValue) { + protected void verifyLastApplied(final TestActorRef shard, final long expectedValue) { long lastApplied = -1; for(int i = 0; i < 20 * 5; i++) { lastApplied = shard.underlyingActor().getShardMBean().getLastApplied(); @@ -179,9 +208,9 @@ public abstract class AbstractShardTest extends AbstractActorTest{ final MutableCompositeModification modification, final Function> preCommit) { - ReadWriteShardDataTreeTransaction tx = dataStore.newReadWriteTransaction("setup-mock-" + cohortName, null); + final ReadWriteShardDataTreeTransaction tx = dataStore.newReadWriteTransaction("setup-mock-" + cohortName, null); tx.getSnapshot().write(path, data); - ShardDataTreeCohort cohort = createDelegatingMockCohort(cohortName, dataStore.finishTransaction(tx), preCommit); + final ShardDataTreeCohort cohort = createDelegatingMockCohort(cohortName, dataStore.finishTransaction(tx), preCommit); modification.addModification(new WriteModification(path, data)); @@ -196,7 +225,7 @@ public abstract class AbstractShardTest extends AbstractActorTest{ protected ShardDataTreeCohort createDelegatingMockCohort(final String cohortName, final ShardDataTreeCohort actual, final Function> preCommit) { - ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, cohortName); + final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, cohortName); doAnswer(new Answer>() { @Override @@ -240,31 +269,111 @@ public abstract class AbstractShardTest extends AbstractActorTest{ return cohort; } - public static NormalizedNode readStore(final TestActorRef shard, final YangInstanceIdentifier id) - throws ExecutionException, InterruptedException { - return readStore(shard.underlyingActor().getDataStore().getDataTree(), id); + protected Object prepareReadyTransactionMessage(boolean remoteReadWriteTransaction, Shard shard, ShardDataTreeCohort cohort, + String transactionID, + MutableCompositeModification modification, + boolean doCommitOnReady) { + if(remoteReadWriteTransaction){ + return prepareForwardedReadyTransaction(cohort, transactionID, CURRENT_VERSION, + doCommitOnReady); + } else { + setupCohortDecorator(shard, cohort); + return prepareBatchedModifications(transactionID, modification, doCommitOnReady); + } } - public static NormalizedNode readStore(final DataTree store, final YangInstanceIdentifier id) { - DataTreeSnapshot transaction = store.takeSnapshot(); + protected ShardDataTreeCohort mockShardDataTreeCohort() { + ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit(); + doReturn(Futures.immediateFuture(null)).when(cohort).preCommit(); + doReturn(Futures.immediateFuture(null)).when(cohort).commit(); + doReturn(mockCandidate("candidate")).when(cohort).getCandidate(); + return cohort; + } + + static ShardDataTreeTransactionParent newShardDataTreeTransactionParent(ShardDataTreeCohort cohort) { + ShardDataTreeTransactionParent mockParent = mock(ShardDataTreeTransactionParent.class); + doReturn(cohort).when(mockParent).finishTransaction(any(ReadWriteShardDataTreeTransaction.class)); + doNothing().when(mockParent).abortTransaction(any(AbstractShardDataTreeTransaction.class)); + return mockParent; + } + + protected ForwardedReadyTransaction prepareForwardedReadyTransaction(ShardDataTreeCohort cohort, + String transactionID, short version, boolean doCommitOnReady) { + return new ForwardedReadyTransaction(transactionID, version, + new ReadWriteShardDataTreeTransaction(newShardDataTreeTransactionParent(cohort), transactionID, + mock(DataTreeModification.class)), doCommitOnReady); + } + + protected Object prepareReadyTransactionMessage(boolean remoteReadWriteTransaction, Shard shard, ShardDataTreeCohort cohort, + String transactionID, + MutableCompositeModification modification) { + return prepareReadyTransactionMessage(remoteReadWriteTransaction, shard, cohort, transactionID, modification, false); + } - Optional> optional = transaction.readNode(id); - NormalizedNode node = optional.isPresent()? optional.get() : null; + protected void setupCohortDecorator(Shard shard, final ShardDataTreeCohort cohort) { + shard.getCommitCoordinator().setCohortDecorator(new ShardCommitCoordinator.CohortDecorator() { + @Override + public ShardDataTreeCohort decorate(Identifier transactionID, ShardDataTreeCohort actual) { + return cohort; + } + }); + } + + protected BatchedModifications prepareBatchedModifications(String transactionID, + MutableCompositeModification modification) { + return prepareBatchedModifications(transactionID, modification, false); + } + + private static BatchedModifications prepareBatchedModifications(String transactionID, + MutableCompositeModification modification, + boolean doCommitOnReady) { + final BatchedModifications batchedModifications = new BatchedModifications(transactionID, CURRENT_VERSION, null); + batchedModifications.addModification(modification); + batchedModifications.setReady(true); + batchedModifications.setDoCommitOnReady(doCommitOnReady); + batchedModifications.setTotalMessagesSent(1); + return batchedModifications; + } - return node; + + public static NormalizedNode readStore(final TestActorRef shard, final YangInstanceIdentifier id) + throws ExecutionException, InterruptedException { + return shard.underlyingActor().getDataStore().readNode(id).orNull(); + } + + public static NormalizedNode readStore(final DataTree store, final YangInstanceIdentifier id) { + return store.takeSnapshot().readNode(id).orNull(); } public static void writeToStore(final TestActorRef shard, final YangInstanceIdentifier id, final NormalizedNode node) throws InterruptedException, ExecutionException { - writeToStore(shard.underlyingActor().getDataStore(), id, node); + Future future = Patterns.ask(shard, newBatchedModifications("tx", id, node, true, true, 1), + new Timeout(5, TimeUnit.SECONDS)); + try { + Await.ready(future, Duration.create(5, TimeUnit.SECONDS)); + } catch(TimeoutException e) { + throw new ExecutionException(e); + } } public static void writeToStore(final ShardDataTree store, final YangInstanceIdentifier id, final NormalizedNode node) throws InterruptedException, ExecutionException { - ReadWriteShardDataTreeTransaction transaction = store.newReadWriteTransaction("writeToStore", null); + final ReadWriteShardDataTreeTransaction transaction = store.newReadWriteTransaction("writeToStore", null); transaction.getSnapshot().write(id, node); - ShardDataTreeCohort cohort = transaction.ready(); + final ShardDataTreeCohort cohort = transaction.ready(); + cohort.canCommit().get(); + cohort.preCommit().get(); + cohort.commit(); + } + + public static void mergeToStore(final ShardDataTree store, final YangInstanceIdentifier id, + final NormalizedNode node) throws InterruptedException, ExecutionException { + final ReadWriteShardDataTreeTransaction transaction = store.newReadWriteTransaction("writeToStore", null); + + transaction.getSnapshot().merge(id, node); + final ShardDataTreeCohort cohort = transaction.ready(); cohort.canCommit().get(); cohort.preCommit().get(); cohort.commit(); @@ -272,7 +381,7 @@ public abstract class AbstractShardTest extends AbstractActorTest{ public static void writeToStore(final DataTree store, final YangInstanceIdentifier id, final NormalizedNode node) throws DataValidationFailedException { - DataTreeModification transaction = store.takeSnapshot().newModification(); + final DataTreeModification transaction = store.takeSnapshot().newModification(); transaction.write(id, node); transaction.ready(); @@ -281,6 +390,84 @@ public abstract class AbstractShardTest extends AbstractActorTest{ store.commit(candidate); } + DataTree setupInMemorySnapshotStore() throws DataValidationFailedException { + final DataTree testStore = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL); + testStore.setSchemaContext(SCHEMA_CONTEXT); + + writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME)); + + final NormalizedNode root = readStore(testStore, YangInstanceIdentifier.EMPTY); + + InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create( + SerializationUtils.serializeNormalizedNode(root), + Collections.emptyList(), 0, 1, -1, -1)); + return testStore; + } + + static DataTreeCandidatePayload payloadForModification(final DataTree source, final DataTreeModification mod) throws DataValidationFailedException { + source.validate(mod); + final DataTreeCandidate candidate = source.prepare(mod); + source.commit(candidate); + return DataTreeCandidatePayload.create(candidate); + } + + static BatchedModifications newBatchedModifications(final String transactionID, final YangInstanceIdentifier path, + final NormalizedNode data, final boolean ready, final boolean doCommitOnReady, final int messagesSent) { + return newBatchedModifications(transactionID, null, path, data, ready, doCommitOnReady, messagesSent); + } + + static BatchedModifications newBatchedModifications(final String transactionID, final String transactionChainID, + final YangInstanceIdentifier path, final NormalizedNode data, final boolean ready, final boolean doCommitOnReady, + final int messagesSent) { + final BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, transactionChainID); + batched.addModification(new WriteModification(path, data)); + batched.setReady(ready); + batched.setDoCommitOnReady(doCommitOnReady); + batched.setTotalMessagesSent(messagesSent); + return batched; + } + + @SuppressWarnings("unchecked") + static void verifyOuterListEntry(final TestActorRef shard, final Object expIDValue) throws Exception { + final NormalizedNode outerList = readStore(shard, TestModel.OUTER_LIST_PATH); + assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList); + assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable", + outerList.getValue() instanceof Iterable); + final Object entry = ((Iterable)outerList.getValue()).iterator().next(); + assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode", + entry instanceof MapEntryNode); + final MapEntryNode mapEntry = (MapEntryNode)entry; + final Optional> idLeaf = + mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME)); + assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent()); + assertEquals(TestModel.ID_QNAME.getLocalName() + " value", expIDValue, idLeaf.get().getValue()); + } + + public static DataTreeCandidateTip mockCandidate(final String name) { + final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name); + final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node"); + doReturn(ModificationType.WRITE).when(mockCandidateNode).getModificationType(); + doReturn(Optional.of(ImmutableNodes.containerNode(CarsModel.CARS_QNAME))).when(mockCandidateNode).getDataAfter(); + doReturn(CarsModel.BASE_PATH).when(mockCandidate).getRootPath(); + doReturn(mockCandidateNode).when(mockCandidate).getRootNode(); + return mockCandidate; + } + + static DataTreeCandidateTip mockUnmodifiedCandidate(final String name) { + final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name); + final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node"); + doReturn(ModificationType.UNMODIFIED).when(mockCandidateNode).getModificationType(); + doReturn(YangInstanceIdentifier.EMPTY).when(mockCandidate).getRootPath(); + doReturn(mockCandidateNode).when(mockCandidate).getRootNode(); + return mockCandidate; + } + + static void commitTransaction(final DataTree store, final DataTreeModification modification) throws DataValidationFailedException { + modification.ready(); + store.validate(modification); + store.commit(store.prepare(modification)); + } + @SuppressWarnings("serial") public static final class DelegatingShardCreator implements Creator { private final Creator delegate;