You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "vldpyatkov (via GitHub)" <gi...@apache.org> on 2023/06/13 07:18:47 UTC

[GitHub] [ignite-3] vldpyatkov opened a new pull request, #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

vldpyatkov opened a new pull request, #2183:
URL: https://github.com/apache/ignite-3/pull/2183

   …geManager.start()


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230602732


##########
modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java:
##########
@@ -118,7 +118,7 @@ void setUp(TestInfo testInfo, @InjectConfiguration RaftConfiguration raftConfigu
         raftManager.start();
         metaStorageManager.start();
 
-        metaStorageManager.deployWatches();
+        assertThat("Watches were not deployed", metaStorageManager.deployWatches(), willCompleteSuccessfully());

Review Comment:
   As I know, we do not have to use dot int the end of sentences in log.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230693349


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+
+    /** Vault manager. */
+    private static VaultManager vaultManager;
+
+    @BeforeAll
+    public static void init() {
+        vaultManager = new VaultManager(new InMemoryVaultService());
+
+        vaultManager.start();
+    }
+
+    @AfterAll
+    public static void deInit() {
+        vaultManager.stop();
+    }
+
+    /**
+     * Returns a stream with test arguments.
+     *
+     * @return Stream of different types of Meta storages to to check.
+     * @throws Exception If failed.
+     */
+    private static Stream<MetaStorageManager> metaStorageProvider() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        return Stream.of(
+                new MetaStorageManagerImpl(
+                        vaultManager,
+                        clusterService,
+                        cmgManager,
+                        logicalTopologyService,
+                        raftManager,
+                        new SimpleInMemoryKeyValueStorage(mcNodeName),
+                        clock
+                ),
+                StandaloneMetaStorageManager.create(vaultManager)
+        );
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     */
+    @ParameterizedTest
+    @MethodSource("metaStorageProvider")
+    public void testCheckCorrectness(MetaStorageManager metastore) {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   No, I add this in the test body.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228652145


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();
 
         // Bump Meta Storage applied revision by modifying a fake key. DistributionZoneManager breaks on start if Vault is not empty, but
         // Meta Storage revision is equal to 0.
         var fakeKey = new ByteArray("foobar");
 
-        CompletableFuture<Boolean> invokeFuture = metaStorageManager.invoke(
+        CompletableFuture<Boolean> invokeFuture = deployWatchesFut.thenCompose(unused -> metaStorageManager.invoke(

Review Comment:
   I think, if before, we have no guarantees that MC start earlier than watches subscribe, it means thus tests are buggy.
   Ok, add synchronous waiting in every invocation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1229137383


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -170,8 +170,10 @@ public interface MetaStorageManager extends IgniteComponent {
      * Starts all registered watches.
      *
      * <p>Should be called after all Ignite components have registered required watches and they are ready to process Meta Storage events.
+     *
+     * @return Future to complete.

Review Comment:
   You're joking right? The core idea of the current ticket (IGNITE-19606) is to linearise ms.start() and ms.deployWatches().
   Do we still have an implementation with unconnected start and deployWatches? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228286531


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.Test;
+
+/**
+ * There are tests of correctness invocation {@link MetaStorageManager#deployWatches()}.

Review Comment:
   It's just "Test that checks correctness of an invocation ..." without "There are".



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228354015


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -170,8 +170,10 @@ public interface MetaStorageManager extends IgniteComponent {
      * Starts all registered watches.
      *
      * <p>Should be called after all Ignite components have registered required watches and they are ready to process Meta Storage events.
+     *
+     * @return Future to complete.

Review Comment:
   Could you provide something what you want to see here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230576501


##########
modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageMultipleNodesAbstractTest.java:
##########
@@ -228,6 +233,8 @@ void testLearnerJoin(TestInfo testInfo) throws NodeStoppingException {
 
         firstNode.cmgManager.initCluster(List.of(firstNode.name()), List.of(firstNode.name()), "test");
 
+        firstNode.deployWatches();

Review Comment:
   It seems incorrect. deployWatches is a part of node start procedure.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230653470


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+
+    /** Vault manager. */
+    private static VaultManager vaultManager;
+
+    @BeforeAll
+    public static void init() {
+        vaultManager = new VaultManager(new InMemoryVaultService());
+
+        vaultManager.start();
+    }
+
+    @AfterAll
+    public static void deInit() {
+        vaultManager.stop();
+    }
+
+    /**
+     * Returns a stream with test arguments.
+     *
+     * @return Stream of different types of Meta storages to to check.
+     * @throws Exception If failed.
+     */
+    private static Stream<MetaStorageManager> metaStorageProvider() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        return Stream.of(
+                new MetaStorageManagerImpl(
+                        vaultManager,
+                        clusterService,
+                        cmgManager,
+                        logicalTopologyService,
+                        raftManager,
+                        new SimpleInMemoryKeyValueStorage(mcNodeName),
+                        clock
+                ),
+                StandaloneMetaStorageManager.create(vaultManager)
+        );
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     */
+    @ParameterizedTest
+    @MethodSource("metaStorageProvider")
+    public void testCheckCorrectness(MetaStorageManager metastore) {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   I added the methods' invocation for MC. But I do not think that have to be implemented in suggested manner.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1229131388


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();
 
         // Bump Meta Storage applied revision by modifying a fake key. DistributionZoneManager breaks on start if Vault is not empty, but
         // Meta Storage revision is equal to 0.
         var fakeKey = new ByteArray("foobar");
 
-        CompletableFuture<Boolean> invokeFuture = metaStorageManager.invoke(
+        CompletableFuture<Boolean> invokeFuture = deployWatchesFut.thenCompose(unused -> metaStorageManager.invoke(

Review Comment:
   > Ok, add synchronous waiting in every invocation.
   Well, in tests I'd rather add assertThat(willSucceedFast()) with proper message in order to have a meaningful explanation if synchronous waiting will hang instead of simple join.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228302585


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.Test;
+
+/**
+ * There are tests of correctness invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+    @Test
+    public void tesMetaStorageManager() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        var metastore = new MetaStorageManagerImpl(
+                new VaultManager(new InMemoryVaultService()),
+                clusterService,
+                cmgManager,
+                logicalTopologyService,
+                raftManager,
+                new SimpleInMemoryKeyValueStorage(mcNodeName),
+                clock
+        );
+
+        checkCorrectness(metastore);
+    }
+
+    @Test
+    public void tesStandaloneMetaStorageManager() throws Exception {
+        var metastore = StandaloneMetaStorageManager.create(new VaultManager(new InMemoryVaultService()));
+
+        checkCorrectness(metastore);
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     * @throws NodeStoppingException If failed.
+     */
+    private static void checkCorrectness(MetaStorageManager metastore) throws NodeStoppingException {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
    metastore.start() is  async thus `deployWatchesFut.isDone()` isn't guaranteed without some awaiting logic. Please use 
   `org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher#willSucceedFast` or similar instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230613375


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java:
##########
@@ -514,6 +516,8 @@ void testDestroyPartitionStoragesOnRestartEvictedNode(TestInfo testInfo) throws
 
         newNode.start();
 
+        newNode.deployWatches();

Review Comment:
   Same as above.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230582451


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java:
##########
@@ -277,14 +278,16 @@ public void unregisterWatch(WatchListener lsnr) {
     }
 
     @Override
-    public void deployWatches() throws NodeStoppingException {
+    public CompletableFuture<Void> deployWatches() {
         if (!busyLock.enterBusy()) {
-            throw new NodeStoppingException();
+            return CompletableFuture.failedFuture(new NodeStoppingException());

Review Comment:
   Good catch!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov merged pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov merged PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228276622


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();
 
         // Bump Meta Storage applied revision by modifying a fake key. DistributionZoneManager breaks on start if Vault is not empty, but
         // Meta Storage revision is equal to 0.
         var fakeKey = new ByteArray("foobar");
 
-        CompletableFuture<Boolean> invokeFuture = metaStorageManager.invoke(
+        CompletableFuture<Boolean> invokeFuture = deployWatchesFut.thenCompose(unused -> metaStorageManager.invoke(

Review Comment:
   Is it the only place we should await deployWatches? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228359724


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();

Review Comment:
   > I think it is possible for test purposes.
   I've never heard about such possibilities for test classes. Is it allowed by our coding conventions?



##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();

Review Comment:
   > I think it is possible for test purposes.
   
   I've never heard about such possibilities for test classes. Is it allowed by our coding conventions?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230692169


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+
+    /** Vault manager. */
+    private static VaultManager vaultManager;
+
+    @BeforeAll
+    public static void init() {
+        vaultManager = new VaultManager(new InMemoryVaultService());
+
+        vaultManager.start();
+    }
+
+    @AfterAll
+    public static void deInit() {
+        vaultManager.stop();
+    }
+
+    /**
+     * Returns a stream with test arguments.
+     *
+     * @return Stream of different types of Meta storages to to check.
+     * @throws Exception If failed.
+     */
+    private static Stream<MetaStorageManager> metaStorageProvider() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        return Stream.of(
+                new MetaStorageManagerImpl(
+                        vaultManager,
+                        clusterService,
+                        cmgManager,
+                        logicalTopologyService,
+                        raftManager,
+                        new SimpleInMemoryKeyValueStorage(mcNodeName),
+                        clock
+                ),
+                StandaloneMetaStorageManager.create(vaultManager)
+        );
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     */
+    @ParameterizedTest
+    @MethodSource("metaStorageProvider")
+    public void testCheckCorrectness(MetaStorageManager metastore) {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   Storage is not a component at all.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230577663


##########
modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java:
##########
@@ -118,7 +118,7 @@ void setUp(TestInfo testInfo, @InjectConfiguration RaftConfiguration raftConfigu
         raftManager.start();
         metaStorageManager.start();
 
-        metaStorageManager.deployWatches();
+        assertThat("Watches were not deployed", metaStorageManager.deployWatches(), willCompleteSuccessfully());

Review Comment:
   Please add full stop at the end of the sentence. Related to all such messages.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228365449


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -170,8 +170,10 @@ public interface MetaStorageManager extends IgniteComponent {
      * Starts all registered watches.
      *
      * <p>Should be called after all Ignite components have registered required watches and they are ready to process Meta Storage events.
+     *
+     * @return Future to complete.

Review Comment:
   Something like: "future that'll completed when meta storage manager is started and deploying watches is finished."



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228270424


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -170,8 +170,10 @@ public interface MetaStorageManager extends IgniteComponent {
      * Starts all registered watches.
      *
      * <p>Should be called after all Ignite components have registered required watches and they are ready to process Meta Storage events.
+     *
+     * @return Future to complete.

Review Comment:
   Please add meaningful comment here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228313489


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.Test;
+
+/**
+ * There are tests of correctness invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+    @Test
+    public void tesMetaStorageManager() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        var metastore = new MetaStorageManagerImpl(
+                new VaultManager(new InMemoryVaultService()),
+                clusterService,
+                cmgManager,
+                logicalTopologyService,
+                raftManager,
+                new SimpleInMemoryKeyValueStorage(mcNodeName),
+                clock
+        );
+
+        checkCorrectness(metastore);
+    }
+
+    @Test
+    public void tesStandaloneMetaStorageManager() throws Exception {
+        var metastore = StandaloneMetaStorageManager.create(new VaultManager(new InMemoryVaultService()));
+
+        checkCorrectness(metastore);
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     * @throws NodeStoppingException If failed.
+     */
+    private static void checkCorrectness(MetaStorageManager metastore) throws NodeStoppingException {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   It is just a unit test, I know the behavior here.
   I am sure we will invoke the listener in the same thread where the start future is completed. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228259420


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();

Review Comment:
   AFAIK it's not valid to use var here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230610405


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java:
##########
@@ -216,6 +223,8 @@ void testRestartWithPds(@WorkDirectory Path workDir, TestInfo testInfo) throws E
         try {
             node2.start();
 
+            node2.deployWatches();

Review Comment:
   Same as above.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230638363


##########
modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageMultipleNodesAbstractTest.java:
##########
@@ -228,6 +233,8 @@ void testLearnerJoin(TestInfo testInfo) throws NodeStoppingException {
 
         firstNode.cmgManager.initCluster(List.of(firstNode.name()), List.of(firstNode.name()), "test");
 
+        firstNode.deployWatches();

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230688779


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -494,24 +492,15 @@ private PartialNode startPartialNode(
                 fut -> new TestConfigurationCatchUpListener(cfgStorage, fut, revisionCallback0)
         );
 
-        CompletableFuture<?> notificationFuture = CompletableFuture.allOf(
+        CompletableFuture<?> startFuture = CompletableFuture.allOf(
                 nodeCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners(),
-                clusterCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners()
+                clusterCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners(),
+                // Deploy all registered watches because all components are ready and have registered their listeners.
+                metaStorageMgr.deployWatches(),
+                configurationCatchUpFuture
         );
 
-        CompletableFuture<?> startFuture = notificationFuture
-                .thenCompose(v -> {
-                    // Deploy all registered watches because all components are ready and have registered their listeners.
-                    try {
-                        metaStorageMgr.deployWatches();
-                    } catch (NodeStoppingException e) {
-                        throw new CompletionException(e);
-                    }
-
-                    return configurationCatchUpFuture;
-                });
-
-        assertThat(startFuture, willCompleteSuccessfully());
+        assertThat("Partial node was not started", startFuture, willCompleteSuccessfully());

Review Comment:
   Really? Seems that originally there was notificationFuture(notifyCurrentConfigurationListeners, notifyCurrentConfigurationListeners).thenCompose(deployWatches) and now it's startFuture(notifyCurrentConfigurationListeners, notifyCurrentConfigurationListeners, deployWatches) in other words originally deployWatches was called after notificationFutures and now it's called concurrently. Am I mistaken?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230616314


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -494,24 +492,15 @@ private PartialNode startPartialNode(
                 fut -> new TestConfigurationCatchUpListener(cfgStorage, fut, revisionCallback0)
         );
 
-        CompletableFuture<?> notificationFuture = CompletableFuture.allOf(
+        CompletableFuture<?> startFuture = CompletableFuture.allOf(
                 nodeCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners(),
-                clusterCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners()
+                clusterCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners(),
+                // Deploy all registered watches because all components are ready and have registered their listeners.
+                metaStorageMgr.deployWatches(),
+                configurationCatchUpFuture
         );
 
-        CompletableFuture<?> startFuture = notificationFuture
-                .thenCompose(v -> {
-                    // Deploy all registered watches because all components are ready and have registered their listeners.
-                    try {
-                        metaStorageMgr.deployWatches();
-                    } catch (NodeStoppingException e) {
-                        throw new CompletionException(e);
-                    }
-
-                    return configurationCatchUpFuture;
-                });
-
-        assertThat(startFuture, willCompleteSuccessfully());
+        assertThat("Partial node was not started", startFuture, willCompleteSuccessfully());

Review Comment:
   Why you've changed the order of the processes?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230681891


##########
modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java:
##########
@@ -118,7 +118,7 @@ void setUp(TestInfo testInfo, @InjectConfiguration RaftConfiguration raftConfigu
         raftManager.start();
         metaStorageManager.start();
 
-        metaStorageManager.deployWatches();
+        assertThat("Watches were not deployed", metaStorageManager.deployWatches(), willCompleteSuccessfully());

Review Comment:
   AFAIK that was revised from Ignite 2 times, and now we follow common grammar rules.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230728268


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+
+    /** Vault manager. */
+    private static VaultManager vaultManager;
+
+    @BeforeAll
+    public static void init() {
+        vaultManager = new VaultManager(new InMemoryVaultService());
+
+        vaultManager.start();
+    }
+
+    @AfterAll
+    public static void deInit() {
+        vaultManager.stop();
+    }
+
+    /**
+     * Returns a stream with test arguments.
+     *
+     * @return Stream of different types of Meta storages to to check.
+     * @throws Exception If failed.
+     */
+    private static Stream<MetaStorageManager> metaStorageProvider() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        return Stream.of(
+                new MetaStorageManagerImpl(
+                        vaultManager,
+                        clusterService,
+                        cmgManager,
+                        logicalTopologyService,
+                        raftManager,
+                        new SimpleInMemoryKeyValueStorage(mcNodeName),
+                        clock
+                ),
+                StandaloneMetaStorageManager.create(vaultManager)
+        );
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     */
+    @ParameterizedTest
+    @MethodSource("metaStorageProvider")
+    public void testCheckCorrectness(MetaStorageManager metastore) {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   Not sure that I got you. I will add, or you've already added? I'm asking because currently I can see only vault's stop.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228369000


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();
 
         // Bump Meta Storage applied revision by modifying a fake key. DistributionZoneManager breaks on start if Vault is not empty, but
         // Meta Storage revision is equal to 0.
         var fakeKey = new ByteArray("foobar");
 
-        CompletableFuture<Boolean> invokeFuture = metaStorageManager.invoke(
+        CompletableFuture<Boolean> invokeFuture = deployWatchesFut.thenCompose(unused -> metaStorageManager.invoke(

Review Comment:
   Well, seems that you've substituted sync method with an async one and thus it's your aim to check whether you break the guaranties in any other places and add chaining or join wherever needed (if needed).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230684148


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+
+    /** Vault manager. */
+    private static VaultManager vaultManager;
+
+    @BeforeAll
+    public static void init() {
+        vaultManager = new VaultManager(new InMemoryVaultService());
+
+        vaultManager.start();
+    }
+
+    @AfterAll
+    public static void deInit() {
+        vaultManager.stop();
+    }
+
+    /**
+     * Returns a stream with test arguments.
+     *
+     * @return Stream of different types of Meta storages to to check.
+     * @throws Exception If failed.
+     */
+    private static Stream<MetaStorageManager> metaStorageProvider() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        return Stream.of(
+                new MetaStorageManagerImpl(
+                        vaultManager,
+                        clusterService,
+                        cmgManager,
+                        logicalTopologyService,
+                        raftManager,
+                        new SimpleInMemoryKeyValueStorage(mcNodeName),
+                        clock
+                ),
+                StandaloneMetaStorageManager.create(vaultManager)
+        );
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     */
+    @ParameterizedTest
+    @MethodSource("metaStorageProvider")
+    public void testCheckCorrectness(MetaStorageManager metastore) {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   Well, did you add meta storage beforeNodeStop/stop? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230975872


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java:
##########
@@ -681,6 +688,8 @@ private class Node {
                     hybridClock
             );
 
+            deployWatchesFut = metaStorageManager.deployWatches();

Review Comment:
   It's not a proper place, deployWatches should be called after all components start, so basically it should be called where it was called originally. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1229131388


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();
 
         // Bump Meta Storage applied revision by modifying a fake key. DistributionZoneManager breaks on start if Vault is not empty, but
         // Meta Storage revision is equal to 0.
         var fakeKey = new ByteArray("foobar");
 
-        CompletableFuture<Boolean> invokeFuture = metaStorageManager.invoke(
+        CompletableFuture<Boolean> invokeFuture = deployWatchesFut.thenCompose(unused -> metaStorageManager.invoke(

Review Comment:
   > Ok, add synchronous waiting in every invocation.
   
   Well, in tests I'd rather add assertThat(willSucceedFast()) with proper message in order to have a meaningful explanation if synchronous waiting will hang instead of simple join.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230576501


##########
modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageMultipleNodesAbstractTest.java:
##########
@@ -228,6 +233,8 @@ void testLearnerJoin(TestInfo testInfo) throws NodeStoppingException {
 
         firstNode.cmgManager.initCluster(List.of(firstNode.name()), List.of(firstNode.name()), "test");
 
+        firstNode.deployWatches();

Review Comment:
   It seems incorrect. deployWatches is a part of node start procedure, that will be completed after cmg init, so that, basically it's still required to call deployWathces within node start and assert it completion after 
   `firstNode.cmgManager.initCluster(List.of(firstNode.name()), List.of(firstNode.name()), "test");`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230636232


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -494,24 +492,15 @@ private PartialNode startPartialNode(
                 fut -> new TestConfigurationCatchUpListener(cfgStorage, fut, revisionCallback0)
         );
 
-        CompletableFuture<?> notificationFuture = CompletableFuture.allOf(
+        CompletableFuture<?> startFuture = CompletableFuture.allOf(
                 nodeCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners(),
-                clusterCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners()
+                clusterCfgMgr.configurationRegistry().notifyCurrentConfigurationListeners(),
+                // Deploy all registered watches because all components are ready and have registered their listeners.
+                metaStorageMgr.deployWatches(),
+                configurationCatchUpFuture
         );
 
-        CompletableFuture<?> startFuture = notificationFuture
-                .thenCompose(v -> {
-                    // Deploy all registered watches because all components are ready and have registered their listeners.
-                    try {
-                        metaStorageMgr.deployWatches();
-                    } catch (NodeStoppingException e) {
-                        throw new CompletionException(e);
-                    }
-
-                    return configurationCatchUpFuture;
-                });
-
-        assertThat(startFuture, willCompleteSuccessfully());
+        assertThat("Partial node was not started", startFuture, willCompleteSuccessfully());

Review Comment:
   I just add a message here, because the assertion relates to deploy watches future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230583163


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+

Review Comment:
   Unnecessary blank line.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228294924


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.Test;
+
+/**
+ * There are tests of correctness invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+    @Test
+    public void tesMetaStorageManager() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        var metastore = new MetaStorageManagerImpl(
+                new VaultManager(new InMemoryVaultService()),
+                clusterService,
+                cmgManager,
+                logicalTopologyService,
+                raftManager,
+                new SimpleInMemoryKeyValueStorage(mcNodeName),
+                clock
+        );
+
+        checkCorrectness(metastore);

Review Comment:
   I'd rather use parametrized test here with method source (MethodSource) that will either return StandaloneMetaStorageManager or General meta storage. In that case checkCorrectness should be refactored to parametrized test itself.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228646350


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -170,8 +170,10 @@ public interface MetaStorageManager extends IgniteComponent {
      * Starts all registered watches.
      *
      * <p>Should be called after all Ignite components have registered required watches and they are ready to process Meta Storage events.
+     *
+     * @return Future to complete.

Review Comment:
   Ok, but I wouldn't write this because it is just an interface. Another implementation might not connect the method with start process.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230609913


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java:
##########
@@ -286,6 +287,8 @@ void setUp(@WorkDirectory Path workDir, TestInfo testInfo) throws Exception {
         Stream.of(firstNode, secondNode).parallel().forEach(Node::start);
 
         firstNode.cmgManager.initCluster(List.of(firstNode.name()), List.of(), "cluster");
+
+        Stream.of(firstNode, secondNode).parallel().forEach(Node::deployWatches);

Review Comment:
   Same as above. It's required to call deployWatches within node's startup process and assert deployWatches' future completion after init.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228373071


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.Test;
+
+/**
+ * There are tests of correctness invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+    @Test
+    public void tesMetaStorageManager() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        var metastore = new MetaStorageManagerImpl(
+                new VaultManager(new InMemoryVaultService()),
+                clusterService,
+                cmgManager,
+                logicalTopologyService,
+                raftManager,
+                new SimpleInMemoryKeyValueStorage(mcNodeName),
+                clock
+        );
+
+        checkCorrectness(metastore);
+    }
+
+    @Test
+    public void tesStandaloneMetaStorageManager() throws Exception {
+        var metastore = StandaloneMetaStorageManager.create(new VaultManager(new InMemoryVaultService()));
+
+        checkCorrectness(metastore);
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     * @throws NodeStoppingException If failed.
+     */
+    private static void checkCorrectness(MetaStorageManager metastore) throws NodeStoppingException {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   Why should we exposes such implementation details (e.g. the one that there's no executors or similar inside metaStorage.start) instead of writing guaranteed code based on willSucceedFast?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228322151


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();
 
         // Bump Meta Storage applied revision by modifying a fake key. DistributionZoneManager breaks on start if Vault is not empty, but
         // Meta Storage revision is equal to 0.
         var fakeKey = new ByteArray("foobar");
 
-        CompletableFuture<Boolean> invokeFuture = metaStorageManager.invoke(
+        CompletableFuture<Boolean> invokeFuture = deployWatchesFut.thenCompose(unused -> metaStorageManager.invoke(

Review Comment:
   I do not know. I add explicit waiting here because it is a common test class, which might be used in different circumstances.
   Other places probably have internal guaranties or should wait to this future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228308032


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();

Review Comment:
   This is a test classe. I think it is possible for test purposes.



##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -313,17 +313,17 @@ public static void deployWatchesAndUpdateMetaStorageRevision(MetaStorageManager
             throws NodeStoppingException, InterruptedException {
         // Watches are deployed before distributionZoneManager start in order to update Meta Storage revision before
         // distributionZoneManager's recovery.
-        metaStorageManager.deployWatches();
+        var deployWatchesFut = metaStorageManager.deployWatches();

Review Comment:
   This is a test class. I think it is possible for test purposes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1228289667


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.Test;
+
+/**
+ * There are tests of correctness invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+    @Test
+    public void tesMetaStorageManager() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        var metastore = new MetaStorageManagerImpl(
+                new VaultManager(new InMemoryVaultService()),

Review Comment:
   VaultManager is an Ignite service, so please either start/stop it or use mock instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230585833


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+
+    /** Vault manager. */
+    private static VaultManager vaultManager;
+
+    @BeforeAll
+    public static void init() {
+        vaultManager = new VaultManager(new InMemoryVaultService());
+
+        vaultManager.start();
+    }
+
+    @AfterAll
+    public static void deInit() {
+        vaultManager.stop();

Review Comment:
   Despite the fact that Vault.beforeNodeStop() is no-op, it's still required to call it before calling vaultManager.stop();



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230607232


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+
+    /** Vault manager. */
+    private static VaultManager vaultManager;
+
+    @BeforeAll
+    public static void init() {
+        vaultManager = new VaultManager(new InMemoryVaultService());
+
+        vaultManager.start();
+    }
+
+    @AfterAll
+    public static void deInit() {
+        vaultManager.stop();
+    }
+
+    /**
+     * Returns a stream with test arguments.
+     *
+     * @return Stream of different types of Meta storages to to check.
+     * @throws Exception If failed.
+     */
+    private static Stream<MetaStorageManager> metaStorageProvider() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        return Stream.of(
+                new MetaStorageManagerImpl(
+                        vaultManager,
+                        clusterService,
+                        cmgManager,
+                        logicalTopologyService,
+                        raftManager,
+                        new SimpleInMemoryKeyValueStorage(mcNodeName),
+                        clock
+                ),
+                StandaloneMetaStorageManager.create(vaultManager)
+        );
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     */
+    @ParameterizedTest
+    @MethodSource("metaStorageProvider")
+    public void testCheckCorrectness(MetaStorageManager metastore) {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   Same as for vault, MetaStorageManager is an IgniteCompoment thus it's required not only to start it but also call beforeNodeStop and stop. All in all, I'd suggest adding vault and ms to some components list and close it (beforeNodeStop + stop). From my point of view, it also may worth to change @Before**All**/@After**All** to Before**Each**/After**Each**.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2183: IGNITE-19606 Linearize metaStorageManager.deployWatches and metaStora…

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2183:
URL: https://github.com/apache/ignite-3/pull/2183#discussion_r1230728268


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageDeployWatchesCorrectnessTest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.network.ClusterService;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests that check correctness of an invocation {@link MetaStorageManager#deployWatches()}.
+ */
+public class MetaStorageDeployWatchesCorrectnessTest extends IgniteAbstractTest {
+
+    /** Vault manager. */
+    private static VaultManager vaultManager;
+
+    @BeforeAll
+    public static void init() {
+        vaultManager = new VaultManager(new InMemoryVaultService());
+
+        vaultManager.start();
+    }
+
+    @AfterAll
+    public static void deInit() {
+        vaultManager.stop();
+    }
+
+    /**
+     * Returns a stream with test arguments.
+     *
+     * @return Stream of different types of Meta storages to to check.
+     * @throws Exception If failed.
+     */
+    private static Stream<MetaStorageManager> metaStorageProvider() throws Exception {
+        HybridClock clock = new HybridClockImpl();
+        String mcNodeName = "mc-node-1";
+
+        ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
+        ClusterService clusterService = mock(ClusterService.class);
+        LogicalTopologyService logicalTopologyService = mock(LogicalTopologyService.class);
+        RaftManager raftManager = mock(RaftManager.class);
+
+        when(cmgManager.metaStorageNodes()).thenReturn(completedFuture(Set.of(mcNodeName)));
+        when(clusterService.nodeName()).thenReturn(mcNodeName);
+        when(raftManager.startRaftGroupNodeAndWaitNodeReadyFuture(any(), any(), any(), any(), any())).thenReturn(completedFuture(null));
+
+        return Stream.of(
+                new MetaStorageManagerImpl(
+                        vaultManager,
+                        clusterService,
+                        cmgManager,
+                        logicalTopologyService,
+                        raftManager,
+                        new SimpleInMemoryKeyValueStorage(mcNodeName),
+                        clock
+                ),
+                StandaloneMetaStorageManager.create(vaultManager)
+        );
+    }
+
+    /**
+     * Invokes {@link MetaStorageManager#deployWatches()} and checks result.
+     *
+     * @param metastore Meta storage.
+     */
+    @ParameterizedTest
+    @MethodSource("metaStorageProvider")
+    public void testCheckCorrectness(MetaStorageManager metastore) {
+        var deployWatchesFut = metastore.deployWatches();
+
+        assertFalse(deployWatchesFut.isDone());
+
+        metastore.start();

Review Comment:
   Not sure that I get you. I will add, or you've already added? I'm asking because currently I can see only vault's stop.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org