You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2023/03/22 10:00:16 UTC

[ignite-3] 01/02: WIP. Introduce StandaloneMetastorageManager for test purposes.

This is an automated email from the ASF dual-hosted git repository.

amashenkov pushed a commit to branch ignite-19080
in repository https://gitbox.apache.org/repos/asf/ignite-3.git

commit fd0bd3850ca68724bf5b86449b22bdadbd710cf1
Author: amashenkov <an...@gmail.com>
AuthorDate: Wed Mar 22 12:41:41 2023 +0300

    WIP. Introduce StandaloneMetastorageManager for test purposes.
---
 modules/metastorage/build.gradle                   |   5 +
 .../metastorage/impl/MetaStorageManagerImpl.java   |   2 +-
 .../impl/StandaloneMetastorageManager.java         | 172 +++++++++++++++++++++
 3 files changed, 178 insertions(+), 1 deletion(-)

diff --git a/modules/metastorage/build.gradle b/modules/metastorage/build.gradle
index fb5bb8c596..3eb17438d7 100644
--- a/modules/metastorage/build.gradle
+++ b/modules/metastorage/build.gradle
@@ -57,9 +57,14 @@ dependencies {
     integrationTestImplementation testFixtures(project(':ignite-metastorage'))
     integrationTestImplementation testFixtures(project(':ignite-cluster-management'))
 
+    testFixturesImplementation project(':ignite-cluster-management')
     testFixturesImplementation project(':ignite-core')
+    testFixturesImplementation project(':ignite-raft-api')
     testFixturesImplementation project(':ignite-rocksdb-common')
+    testFixturesImplementation project(':ignite-vault')
+    testFixturesImplementation testFixtures(project(':ignite-vault'))
     testFixturesImplementation libs.jetbrains.annotations
+    testFixturesImplementation libs.mockito.core
 }
 
 description = 'ignite-metastorage'
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
index 23452efed6..ea8f5418f0 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
@@ -127,7 +127,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
         this.storage = storage;
     }
 
-    private CompletableFuture<MetaStorageServiceImpl> initializeMetaStorage(Set<String> metaStorageNodes) {
+    protected CompletableFuture<MetaStorageServiceImpl> initializeMetaStorage(Set<String> metaStorageNodes) {
         ClusterNode thisNode = clusterService.topologyService().localMember();
 
         String thisNodeName = thisNode.name();
diff --git a/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/impl/StandaloneMetastorageManager.java b/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/impl/StandaloneMetastorageManager.java
new file mode 100644
index 0000000000..6adb379f50
--- /dev/null
+++ b/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/impl/StandaloneMetastorageManager.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.metastorage.impl;
+
+import static java.util.Collections.singleton;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+
+import java.io.Serializable;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
+import org.apache.ignite.internal.raft.Command;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.raft.ReadCommand;
+import org.apache.ignite.internal.raft.WriteCommand;
+import org.apache.ignite.internal.raft.service.CommandClosure;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.jetbrains.annotations.Nullable;
+import org.mockito.ArgumentMatchers;
+import org.mockito.Mockito;
+
+/**
+ * MetaStorageManager dummy implementation.
+ */
+public class StandaloneMetastorageManager extends MetaStorageManagerImpl {
+    private static final String TEST_NODE_NAME = "test";
+    private final MetaStorageServiceImpl metaStorageService;
+
+    /**
+     * Creates standalone MetaStorage manager.
+     */
+    public static MetaStorageManager create() {
+        VaultManager vaultMgr = new VaultManager(new InMemoryVaultService());
+        SimpleInMemoryKeyValueStorage storage = new SimpleInMemoryKeyValueStorage(TEST_NODE_NAME);
+
+        ClusterManagementGroupManager cmgMgr = Mockito.mock(ClusterManagementGroupManager.class);
+        Mockito.when(cmgMgr.metaStorageNodes()).thenReturn(completedFuture(Set.of())).getMock();
+
+        return new StandaloneMetastorageManager(vaultMgr, cmgMgr, storage) {
+            @Override
+            public void start() {
+                vaultMgr.start();
+                storage.start();
+
+                super.start();
+            }
+
+            @Override
+            public void stop() throws Exception {
+                super.stop();
+                vaultMgr.stop();
+            }
+        };
+    }
+
+    /**
+     * The constructor.
+     *
+     * @param vaultMgr Vault manager.
+     * @param cmgMgr Cluster management group manager.
+     * @param storage Storage.
+     */
+    protected StandaloneMetastorageManager(
+            VaultManager vaultMgr,
+            ClusterManagementGroupManager cmgMgr,
+            KeyValueStorage storage
+    ) {
+        super(vaultMgr, Mockito.mock(ClusterService.class), cmgMgr, Mockito.mock(LogicalTopologyService.class),
+                Mockito.mock(RaftManager.class), storage);
+
+        MetaStorageListener metaStorageListener = new MetaStorageListener(storage);
+
+        RaftGroupService raftGroupService = Mockito.mock(RaftGroupService.class);
+        Mockito.when(raftGroupService.run(ArgumentMatchers.any()))
+                .thenAnswer(invocation -> runCommand(invocation.getArgument(0), metaStorageListener));
+
+        var localNode = new ClusterNode(TEST_NODE_NAME, TEST_NODE_NAME, new NetworkAddress("localhost", 10000));
+
+        this.metaStorageService = new MetaStorageServiceImpl(raftGroupService, new IgniteSpinBusyLock(), localNode);
+    }
+
+    @Override
+    protected CompletableFuture<MetaStorageServiceImpl> initializeMetaStorage(Set<String> metaStorageNodes) {
+        return completedFuture(metaStorageService);
+    }
+
+    private static CompletableFuture<Serializable> runCommand(Command command, MetaStorageListener listener) {
+        AtomicReference<CompletableFuture<Serializable>> resRef = new AtomicReference<>();
+
+        CommandClosure<? extends Command> closure = new CommandClosure<>() {
+            @Override
+            public Command command() {
+                return command;
+            }
+
+            @Override
+            public void result(@Nullable Serializable res) {
+                resRef.set(res instanceof Throwable ? failedFuture((Throwable) res) : completedFuture(res));
+            }
+        };
+
+        if (command instanceof ReadCommand) {
+            listener.onRead(singleton((CommandClosure<ReadCommand>) closure).iterator());
+        } else {
+            listener.onWrite(singleton((CommandClosure<WriteCommand>) closure).iterator());
+        }
+
+        return resRef.get();
+    }
+}