Add JournalSegmentFile.map()
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / IntegrationTestKit.java
index b33636522178425c699576d67f394c88e54af5f7..63e9ba72a74fb2606477d6a9caabe44ad7975abf 100644 (file)
@@ -9,7 +9,11 @@ package org.opendaylight.controller.cluster.datastore;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
@@ -17,38 +21,36 @@ import akka.cluster.Cluster;
 import akka.cluster.ClusterEvent.CurrentClusterState;
 import akka.cluster.Member;
 import akka.cluster.MemberStatus;
-import com.google.common.base.Optional;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
-import org.mockito.Mockito;
+import org.opendaylight.controller.cluster.databroker.ClientBackedDataStore;
 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
 import org.opendaylight.controller.cluster.datastore.config.Configuration;
 import org.opendaylight.controller.cluster.datastore.config.ConfigurationImpl;
-import org.opendaylight.controller.cluster.datastore.config.EmptyModuleShardConfigProvider;
-import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
 import org.opendaylight.controller.cluster.datastore.messages.OnDemandShardState;
 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
-import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.cluster.datastore.utils.ActorUtils;
 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
-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.dom.spi.store.DOMStoreReadTransaction;
+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.NormalizedNode;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
-import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
 
 public class IntegrationTestKit extends ShardTestKit {
 
@@ -56,82 +58,92 @@ public class IntegrationTestKit extends ShardTestKit {
 
     protected DatastoreContext.Builder datastoreContextBuilder;
     protected DatastoreSnapshot restoreFromSnapshot;
+    private final int commitTimeout;
 
     public IntegrationTestKit(final ActorSystem actorSystem, final Builder datastoreContextBuilder) {
+        this(actorSystem, datastoreContextBuilder, 7);
+    }
+
+    public IntegrationTestKit(final ActorSystem actorSystem, final Builder datastoreContextBuilder,
+            final int commitTimeout) {
         super(actorSystem);
         this.datastoreContextBuilder = datastoreContextBuilder;
+        this.commitTimeout = commitTimeout;
     }
 
     public DatastoreContext.Builder getDatastoreContextBuilder() {
         return datastoreContextBuilder;
     }
 
-    public AbstractDataStore setupDistributedDataStore(final String typeName, final String... shardNames) {
-        return setupDistributedDataStore(typeName, "module-shards.conf", true, SchemaContextHelper.full(), shardNames);
+    public ClientBackedDataStore setupDataStore(final Class<? extends ClientBackedDataStore> implementation,
+            final String typeName, final String... shardNames) throws Exception {
+        return setupDataStore(implementation, typeName, "module-shards.conf", true, SchemaContextHelper.full(),
+            shardNames);
     }
 
-    public AbstractDataStore setupDistributedDataStore(final String typeName, final boolean waitUntilLeader,
-            final String... shardNames) {
-        return setupDistributedDataStore(typeName, "module-shards.conf", waitUntilLeader,
-                SchemaContextHelper.full(), shardNames);
+    public ClientBackedDataStore setupDataStore(final Class<? extends ClientBackedDataStore> implementation,
+            final String typeName, final boolean waitUntilLeader, final String... shardNames) throws Exception {
+        return setupDataStore(implementation, typeName, "module-shards.conf", waitUntilLeader,
+            SchemaContextHelper.full(), shardNames);
     }
 
-    public AbstractDataStore setupDistributedDataStore(final String typeName, final String moduleShardsConfig,
-            final boolean waitUntilLeader, final String... shardNames) {
-        return setupDistributedDataStore(typeName, moduleShardsConfig, waitUntilLeader,
-                SchemaContextHelper.full(), shardNames);
+    public ClientBackedDataStore setupDataStore(final Class<? extends ClientBackedDataStore> implementation,
+            final String typeName, final String moduleShardsConfig, final boolean waitUntilLeader,
+            final String... shardNames) throws Exception {
+        return setupDataStore(implementation, typeName, moduleShardsConfig, waitUntilLeader,
+            SchemaContextHelper.full(), shardNames);
     }
 
-    public AbstractDataStore setupDistributedDataStore(final String typeName, final String moduleShardsConfig,
-            final boolean waitUntilLeader, final SchemaContext schemaContext, final String... shardNames) {
+    public ClientBackedDataStore setupDataStore(final Class<? extends ClientBackedDataStore> implementation,
+            final String typeName, final String moduleShardsConfig, final boolean waitUntilLeader,
+            final EffectiveModelContext schemaContext, final String... shardNames) throws Exception {
+        return setupDataStore(implementation, typeName, moduleShardsConfig, "modules.conf", waitUntilLeader,
+            schemaContext, shardNames);
+    }
+
+    private ClientBackedDataStore setupDataStore(final Class<? extends ClientBackedDataStore> implementation,
+            final String typeName, final String moduleShardsConfig, final String modulesConfig,
+            final boolean waitUntilLeader, final EffectiveModelContext schemaContext, final String... shardNames)
+                throws Exception {
         final ClusterWrapper cluster = new ClusterWrapperImpl(getSystem());
-        final Configuration config = new ConfigurationImpl(moduleShardsConfig, "modules.conf");
+        final Configuration config = new ConfigurationImpl(moduleShardsConfig, modulesConfig);
+
+        setDataStoreName(typeName);
 
-        datastoreContextBuilder.dataStoreName(typeName);
+        final DatastoreContext datastoreContext = datastoreContextBuilder.build();
+        final DatastoreContextFactory mockContextFactory = mock(DatastoreContextFactory.class);
+        doReturn(datastoreContext).when(mockContextFactory).getBaseDatastoreContext();
+        doReturn(datastoreContext).when(mockContextFactory).getShardDatastoreContext(anyString());
 
-        DatastoreContext datastoreContext = datastoreContextBuilder.build();
-        DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
-        Mockito.doReturn(datastoreContext).when(mockContextFactory).getBaseDatastoreContext();
-        Mockito.doReturn(datastoreContext).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
+        final var constructor = implementation.getDeclaredConstructor(ActorSystem.class, ClusterWrapper.class,
+            Configuration.class, DatastoreContextFactory.class, DatastoreSnapshot.class);
 
-        AbstractDataStore dataStore = new DistributedDataStore(getSystem(), cluster, config, mockContextFactory,
-                restoreFromSnapshot);
+        final var dataStore = constructor.newInstance(getSystem(), cluster, config, mockContextFactory,
+            restoreFromSnapshot);
 
-        dataStore.onGlobalContextUpdated(schemaContext);
+        dataStore.onModelContextUpdated(schemaContext);
 
         if (waitUntilLeader) {
-            waitUntilLeader(dataStore.getActorContext(), shardNames);
+            waitUntilLeader(dataStore.getActorUtils(), shardNames);
         }
 
         datastoreContextBuilder = DatastoreContext.newBuilderFrom(datastoreContext);
         return dataStore;
     }
 
-    public DistributedDataStore setupDistributedDataStoreWithoutConfig(final String typeName,
-                                                                       final SchemaContext schemaContext) {
-        final ClusterWrapper cluster = new ClusterWrapperImpl(getSystem());
-        final ConfigurationImpl configuration = new ConfigurationImpl(new EmptyModuleShardConfigProvider());
-
-        getDatastoreContextBuilder().dataStoreName(typeName);
-
-        final DatastoreContext datastoreContext = getDatastoreContextBuilder().build();
-
-        final DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
-        Mockito.doReturn(datastoreContext).when(mockContextFactory).getBaseDatastoreContext();
-        Mockito.doReturn(datastoreContext).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
-
-        final DistributedDataStore dataStore = new DistributedDataStore(getSystem(), cluster,
-                configuration, mockContextFactory, restoreFromSnapshot);
-
-        dataStore.onGlobalContextUpdated(schemaContext);
-
-        datastoreContextBuilder = DatastoreContext.newBuilderFrom(datastoreContext);
-        return dataStore;
+    private void setDataStoreName(final String typeName) {
+        if ("config".equals(typeName)) {
+            datastoreContextBuilder.logicalStoreType(LogicalDatastoreType.CONFIGURATION);
+        } else if ("operational".equals(typeName)) {
+            datastoreContextBuilder.logicalStoreType(LogicalDatastoreType.OPERATIONAL);
+        } else {
+            datastoreContextBuilder.dataStoreName(typeName);
+        }
     }
 
-    public void waitUntilLeader(final ActorContext actorContext, final String... shardNames) {
+    public void waitUntilLeader(final ActorUtils actorUtils, final String... shardNames) {
         for (String shardName: shardNames) {
-            ActorRef shard = findLocalShard(actorContext, shardName);
+            ActorRef shard = findLocalShard(actorUtils, shardName);
 
             assertNotNull("Shard was not created for " + shardName, shard);
 
@@ -139,9 +151,9 @@ public class IntegrationTestKit extends ShardTestKit {
         }
     }
 
-    public void waitUntilNoLeader(final ActorContext actorContext, final String... shardNames) {
+    public void waitUntilNoLeader(final ActorUtils actorUtils, final String... shardNames) {
         for (String shardName: shardNames) {
-            ActorRef shard = findLocalShard(actorContext, shardName);
+            ActorRef shard = findLocalShard(actorUtils, shardName);
             assertNotNull("No local shard found for " + shardName, shard);
 
             waitUntilNoLeader(shard);
@@ -166,23 +178,22 @@ public class IntegrationTestKit extends ShardTestKit {
         fail("Member(s) " + otherMembersSet + " are not Up");
     }
 
-    public static ActorRef findLocalShard(final ActorContext actorContext, final String shardName) {
-        ActorRef shard = null;
-        for (int i = 0; i < 20 * 5 && shard == null; i++) {
+    public static ActorRef findLocalShard(final ActorUtils actorUtils, final String shardName) {
+        for (int i = 0; i < 20 * 5; i++) {
             Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
-            Optional<ActorRef> shardReply = actorContext.findLocalShard(shardName);
+            Optional<ActorRef> shardReply = actorUtils.findLocalShard(shardName);
             if (shardReply.isPresent()) {
-                shard = shardReply.get();
+                return shardReply.orElseThrow();
             }
         }
-        return shard;
+        return null;
     }
 
-    public static void waitUntilShardIsDown(ActorContext actorContext, String shardName) {
+    public static void waitUntilShardIsDown(final ActorUtils actorUtils, final String shardName) {
         for (int i = 0; i < 20 * 5 ; i++) {
             LOG.debug("Waiting for shard down {}", shardName);
             Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
-            Optional<ActorRef> shardReply = actorContext.findLocalShard(shardName);
+            Optional<ActorRef> shardReply = actorUtils.findLocalShard(shardName);
             if (!shardReply.isPresent()) {
                 return;
             }
@@ -191,17 +202,17 @@ public class IntegrationTestKit extends ShardTestKit {
         throw new IllegalStateException("Shard[" + shardName + " did not shutdown in time");
     }
 
-    public static void verifyShardStats(final AbstractDataStore datastore, final String shardName,
+    public static void verifyShardStats(final ClientBackedDataStore datastore, final String shardName,
             final ShardStatsVerifier verifier) throws Exception {
-        ActorContext actorContext = datastore.getActorContext();
+        ActorUtils actorUtils = datastore.getActorUtils();
 
-        Future<ActorRef> future = actorContext.findLocalShardAsync(shardName);
-        ActorRef shardActor = Await.result(future, Duration.create(10, TimeUnit.SECONDS));
+        Future<ActorRef> future = actorUtils.findLocalShardAsync(shardName);
+        ActorRef shardActor = Await.result(future, FiniteDuration.create(10, TimeUnit.SECONDS));
 
         AssertionError lastError = null;
         Stopwatch sw = Stopwatch.createStarted();
         while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
-            ShardStats shardStats = (ShardStats)actorContext
+            ShardStats shardStats = (ShardStats)actorUtils
                     .executeOperation(shardActor, Shard.GET_SHARD_MBEAN_MESSAGE);
 
             try {
@@ -216,17 +227,17 @@ public class IntegrationTestKit extends ShardTestKit {
         throw lastError;
     }
 
-    public static void verifyShardState(final AbstractDataStore datastore, final String shardName,
+    public static void verifyShardState(final ClientBackedDataStore datastore, final String shardName,
             final Consumer<OnDemandShardState> verifier) throws Exception {
-        ActorContext actorContext = datastore.getActorContext();
+        ActorUtils actorUtils = datastore.getActorUtils();
 
-        Future<ActorRef> future = actorContext.findLocalShardAsync(shardName);
-        ActorRef shardActor = Await.result(future, Duration.create(10, TimeUnit.SECONDS));
+        Future<ActorRef> future = actorUtils.findLocalShardAsync(shardName);
+        ActorRef shardActor = Await.result(future, FiniteDuration.create(10, TimeUnit.SECONDS));
 
         AssertionError lastError = null;
         Stopwatch sw = Stopwatch.createStarted();
         while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
-            OnDemandShardState shardState = (OnDemandShardState)actorContext
+            OnDemandShardState shardState = (OnDemandShardState)actorUtils
                     .executeOperation(shardActor, GetOnDemandRaftState.INSTANCE);
 
             try {
@@ -241,8 +252,8 @@ public class IntegrationTestKit extends ShardTestKit {
         throw lastError;
     }
 
-    void testWriteTransaction(final AbstractDataStore dataStore, final YangInstanceIdentifier nodePath,
-            final NormalizedNode<?, ?> nodeToWrite) throws Exception {
+    void testWriteTransaction(final ClientBackedDataStore dataStore, final YangInstanceIdentifier nodePath,
+            final NormalizedNode nodeToWrite) throws Exception {
 
         // 1. Create a write-only Tx
 
@@ -264,54 +275,29 @@ public class IntegrationTestKit extends ShardTestKit {
         // 5. Verify the data in the store
 
         DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
-
-        Optional<NormalizedNode<?, ?>> optional = readTx.read(nodePath).get(5, TimeUnit.SECONDS);
-        assertEquals("isPresent", true, optional.isPresent());
-        assertEquals("Data node", nodeToWrite, optional.get());
+        assertEquals(Optional.of(nodeToWrite), readTx.read(nodePath).get(5, TimeUnit.SECONDS));
     }
 
     public void doCommit(final DOMStoreThreePhaseCommitCohort cohort) throws Exception {
-        Boolean canCommit = cohort.canCommit().get(7, TimeUnit.SECONDS);
-        assertEquals("canCommit", true, canCommit);
+        Boolean canCommit = cohort.canCommit().get(commitTimeout, TimeUnit.SECONDS);
+        assertEquals("canCommit", Boolean.TRUE, canCommit);
         cohort.preCommit().get(5, TimeUnit.SECONDS);
         cohort.commit().get(5, TimeUnit.SECONDS);
     }
 
     void doCommit(final ListenableFuture<Boolean> canCommitFuture, final DOMStoreThreePhaseCommitCohort cohort)
             throws Exception {
-        Boolean canCommit = canCommitFuture.get(7, TimeUnit.SECONDS);
-        assertEquals("canCommit", true, canCommit);
+        Boolean canCommit = canCommitFuture.get(commitTimeout, TimeUnit.SECONDS);
+        assertEquals("canCommit", Boolean.TRUE, canCommit);
         cohort.preCommit().get(5, TimeUnit.SECONDS);
         cohort.commit().get(5, TimeUnit.SECONDS);
     }
 
-    @SuppressWarnings("checkstyle:IllegalCatch")
-    void assertExceptionOnCall(final Callable<Void> callable, final Class<? extends Exception> expType)
-            throws Exception {
-        try {
-            callable.call();
-            fail("Expected " + expType.getSimpleName());
-        } catch (Exception e) {
-            assertEquals("Exception type", expType, e.getClass());
-        }
-    }
-
     void assertExceptionOnTxChainCreates(final DOMStoreTransactionChain txChain,
-            final Class<? extends Exception> expType) throws Exception {
-        assertExceptionOnCall(() -> {
-            txChain.newWriteOnlyTransaction();
-            return null;
-        }, expType);
-
-        assertExceptionOnCall(() -> {
-            txChain.newReadWriteTransaction();
-            return null;
-        }, expType);
-
-        assertExceptionOnCall(() -> {
-            txChain.newReadOnlyTransaction();
-            return null;
-        }, expType);
+            final Class<? extends Exception> expType) {
+        assertThrows(expType, () -> txChain.newWriteOnlyTransaction());
+        assertThrows(expType, () -> txChain.newReadWriteTransaction());
+        assertThrows(expType, () -> txChain.newReadOnlyTransaction());
     }
 
     public interface ShardStatsVerifier {