You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "alievmirza (via GitHub)" <gi...@apache.org> on 2023/05/30 22:44:52 UTC

[GitHub] [ignite-3] alievmirza opened a new pull request, #2122: IGNITE-18963 Altering filters must recalculate data nodes

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

   https://issues.apache.org/jira/browse/IGNITE-18963


-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests for default zone that node that was added before altering filter is taken into account
+     * after altering of a filter and corresponding immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterForDefaultZone() throws Exception {
+        topology.putNode(A);
+        topology.putNode(B);
+        topology.putNode(C);
+
+        startDistributionZoneManager();
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                DEFAULT_ZONE_ID,
+                Set.of(A, C).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes were changed.
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                DEFAULT_ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests that altering of a filter affects only scale up timers and only added nodes.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterDoNotAffectScaleDown() throws Exception {
+        preparePrerequisites(IMMEDIATE_TIMER_VALUE, 10000);
+
+        topology.putNode(D);
+
+        assertNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleDownTask());
+
+        topology.removeNodes(Set.of(C));
+
+        // Check that scale down task was scheduled.
+        assertTrue(waitForCondition(() -> distributionZoneManager.zonesState().get(ZONE_ID).scaleDownTask() != null, 10_000));
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        // Node C is still in data nodes because altering a filter triggers only immediate scale up.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Check that scale down task is still scheduled.
+        assertNotNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask());
+
+        // Alter zone so we could check that node C is removed from data nodes.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests that node, that was added after altering of a filter (meaning, that immediate scale up was triggered), is added to data nodes
+     * only after altering and corresponding scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testNodeAddedWhileAlteringFilter() throws Exception {
+        preparePrerequisites(10000, INFINITE_TIMER_VALUE);
+
+        assertNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask());
+
+        topology.putNode(D);
+
+        // Check that scale up task was scheduled.
+        assertTrue(waitForCondition(() -> distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask() != null, 10_000));
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        LogicalNode e = new LogicalNode(
+                new ClusterNode("5", "E", new NetworkAddress("localhost", 123)),
+                Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+        );
+
+        doAnswer(invocation -> {
+            If iif = invocation.getArgument(0);
+
+            // Emulate a situation when immediate timer was run after filter altering and new node was added, so timer was scheduled.
+            byte[] key = zoneScaleUpChangeTriggerKey(ZONE_ID).bytes();
+
+            if (Arrays.stream(iif.cond().keys()).anyMatch(k -> Arrays.equals(key, k))) {
+                assertNotNull(distributionZoneManager.zonesState().get(1).scaleUpTask());
+
+                topology.putNode(e);
+            }
+            return invocation.callRealMethod();
+        }).when(keyValueStorage).invoke(any(), any());
+
+        // Check that node E, that was added while filter's altering, is not propagated to data nodes.
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Assert that scheduled timer was not canceled because of immediate scale up after filter altering.
+        assertNotNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask());
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // Check that node E, that was added after filter's altering, was added only after altering immediate scale up.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D, e).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }

Review Comment:
   Some tests are missing, e.g. altering from something-matches to nothing-matches.



-- 
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] alievmirza commented on a diff in pull request #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests for default zone that node that was added before altering filter is taken into account
+     * after altering of a filter and corresponding immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterForDefaultZone() throws Exception {
+        topology.putNode(A);
+        topology.putNode(B);
+        topology.putNode(C);
+
+        startDistributionZoneManager();
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                DEFAULT_ZONE_ID,
+                Set.of(A, C).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes were changed.
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                DEFAULT_ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests that altering of a filter affects only scale up timers and only added nodes.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterDoNotAffectScaleDown() throws Exception {
+        preparePrerequisites(IMMEDIATE_TIMER_VALUE, 10000);
+
+        topology.putNode(D);
+
+        assertNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleDownTask());
+
+        topology.removeNodes(Set.of(C));
+
+        // Check that scale down task was scheduled.
+        assertTrue(waitForCondition(() -> distributionZoneManager.zonesState().get(ZONE_ID).scaleDownTask() != null, 10_000));
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        // Node C is still in data nodes because altering a filter triggers only immediate scale up.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Check that scale down task is still scheduled.
+        assertNotNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask());
+
+        // Alter zone so we could check that node C is removed from data nodes.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests that node, that was added after altering of a filter (meaning, that immediate scale up was triggered), is added to data nodes
+     * only after altering and corresponding scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testNodeAddedWhileAlteringFilter() throws Exception {
+        preparePrerequisites(10000, INFINITE_TIMER_VALUE);

Review Comment:
   done



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests for default zone that node that was added before altering filter is taken into account
+     * after altering of a filter and corresponding immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterForDefaultZone() throws Exception {
+        topology.putNode(A);
+        topology.putNode(B);
+        topology.putNode(C);
+
+        startDistributionZoneManager();
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                DEFAULT_ZONE_ID,
+                Set.of(A, C).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes were changed.
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                DEFAULT_ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests that altering of a filter affects only scale up timers and only added nodes.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterDoNotAffectScaleDown() throws Exception {
+        preparePrerequisites(IMMEDIATE_TIMER_VALUE, 10000);
+
+        topology.putNode(D);
+
+        assertNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleDownTask());
+
+        topology.removeNodes(Set.of(C));
+
+        // Check that scale down task was scheduled.
+        assertTrue(waitForCondition(() -> distributionZoneManager.zonesState().get(ZONE_ID).scaleDownTask() != null, 10_000));
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        // Node C is still in data nodes because altering a filter triggers only immediate scale up.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Check that scale down task is still scheduled.
+        assertNotNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask());
+
+        // Alter zone so we could check that node C is removed from data nodes.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests that node, that was added after altering of a filter (meaning, that immediate scale up was triggered), is added to data nodes
+     * only after altering and corresponding scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testNodeAddedWhileAlteringFilter() throws Exception {
+        preparePrerequisites(10000, INFINITE_TIMER_VALUE);
+
+        assertNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask());
+
+        topology.putNode(D);
+
+        // Check that scale up task was scheduled.
+        assertTrue(waitForCondition(() -> distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask() != null, 10_000));
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        LogicalNode e = new LogicalNode(
+                new ClusterNode("5", "E", new NetworkAddress("localhost", 123)),
+                Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+        );
+
+        doAnswer(invocation -> {
+            If iif = invocation.getArgument(0);
+
+            // Emulate a situation when immediate timer was run after filter altering and new node was added, so timer was scheduled.
+            byte[] key = zoneScaleUpChangeTriggerKey(ZONE_ID).bytes();
+
+            if (Arrays.stream(iif.cond().keys()).anyMatch(k -> Arrays.equals(key, k))) {
+                assertNotNull(distributionZoneManager.zonesState().get(1).scaleUpTask());
+
+                topology.putNode(e);
+            }
+            return invocation.callRealMethod();
+        }).when(keyValueStorage).invoke(any(), any());
+
+        // Check that node E, that was added while filter's altering, is not propagated to data nodes.
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Assert that scheduled timer was not canceled because of immediate scale up after filter altering.
+        assertNotNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask());
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // Check that node E, that was added after filter's altering, was added only after altering immediate scale up.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D, e).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }

Review Comment:
   added



-- 
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] alievmirza commented on a diff in pull request #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";

Review Comment:
   done



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);

Review Comment:
   done



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests for default zone that node that was added before altering filter is taken into account
+     * after altering of a filter and corresponding immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterForDefaultZone() throws Exception {
+        topology.putNode(A);
+        topology.putNode(B);
+        topology.putNode(C);
+
+        startDistributionZoneManager();
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,

Review Comment:
   done



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {

Review Comment:
   I think yes, given test is a right place to have such checks, because it relates DZM internal logic and thus should be checked within corresponding unit test, this one.



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests for default zone that node that was added before altering filter is taken into account
+     * after altering of a filter and corresponding immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterForDefaultZone() throws Exception {
+        topology.putNode(A);
+        topology.putNode(B);
+        topology.putNode(C);
+
+        startDistributionZoneManager();
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,

Review Comment:
   I'd rather use parametrized test for default/custom zone variations.



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);

Review Comment:
   10_000 widely used, let's extract it to constant.



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {

Review Comment:
   Seems that, tests for altering filters though DDL are missing.



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -175,54 +164,64 @@ void testAlterFilterForDefaultZone() throws Exception {
      *
      * @throws Exception If failed.
      */
-    @Test
-    void testAlterFilterDoNotAffectScaleDown() throws Exception {
-        preparePrerequisites(IMMEDIATE_TIMER_VALUE, 10000);
+    @ParameterizedTest
+    @ValueSource(booleans =  {true, false})

Review Comment:
   What about using zone name and zone id as a parameters instead of boolean?



-- 
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 merged pull request #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -175,54 +164,64 @@ void testAlterFilterForDefaultZone() throws Exception {
      *
      * @throws Exception If failed.
      */
-    @Test
-    void testAlterFilterDoNotAffectScaleDown() throws Exception {
-        preparePrerequisites(IMMEDIATE_TIMER_VALUE, 10000);
+    @ParameterizedTest
+    @ValueSource(booleans =  {true, false})

Review Comment:
   What about using zone itself as a parameter instead of boolean?



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests for default zone that node that was added before altering filter is taken into account
+     * after altering of a filter and corresponding immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterForDefaultZone() throws Exception {
+        topology.putNode(A);
+        topology.putNode(B);
+        topology.putNode(C);
+
+        startDistributionZoneManager();
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                DEFAULT_ZONE_ID,
+                Set.of(A, C).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes were changed.
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(
+                DEFAULT_ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(DEFAULT_ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                DEFAULT_ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests that altering of a filter affects only scale up timers and only added nodes.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilterDoNotAffectScaleDown() throws Exception {
+        preparePrerequisites(IMMEDIATE_TIMER_VALUE, 10000);
+
+        topology.putNode(D);
+
+        assertNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleDownTask());
+
+        topology.removeNodes(Set.of(C));
+
+        // Check that scale down task was scheduled.
+        assertTrue(waitForCondition(() -> distributionZoneManager.zonesState().get(ZONE_ID).scaleDownTask() != null, 10_000));
+
+        String newFilter = "$[?(@.region == 'CN')]";
+
+        distributionZoneManager.alterZone(ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+        // Node C is still in data nodes because altering a filter triggers only immediate scale up.
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(C, D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+
+        // Check that scale down task is still scheduled.
+        assertNotNull(distributionZoneManager.zonesState().get(ZONE_ID).scaleUpTask());
+
+        // Alter zone so we could check that node C is removed from data nodes.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                        .filter(newFilter)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        assertDataNodesFromManager(
+                distributionZoneManager,
+                ZONE_ID,
+                Set.of(D).stream().map(ClusterNode::name).collect(Collectors.toSet()),
+                10_000
+        );
+    }
+
+    /**
+     * Tests that node, that was added after altering of a filter (meaning, that immediate scale up was triggered), is added to data nodes
+     * only after altering and corresponding scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testNodeAddedWhileAlteringFilter() throws Exception {
+        preparePrerequisites(10000, INFINITE_TIMER_VALUE);

Review Comment:
   Here and there, 10000 -> 10_000 -> constant.



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";

Review Comment:
   Minor-minor: it might be useful to have a comment explaining which nodes match given filer, e.g.
   ```
   // Nodes C and D matches the filter.
   String newFilter = "$[?(@.region == 'CN')]";
   ```



-- 
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] alievmirza commented on a diff in pull request #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {

Review Comment:
   Do we really need to check this scenrario in this ticket? I mean we've tested here API from `DistributionZoneManager`, and altering tests must be done in the ticket, where altering through DDL was implemented, so they properly call our API method from `DistributionZoneManager`



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {

Review Comment:
   Do we really need to check this scenario in this ticket? I mean we've tested here API from `DistributionZoneManager`, and altering tests must be done in the ticket, where altering through DDL was implemented, so they properly call our API method from `DistributionZoneManager`



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -443,4 +443,34 @@ public static <T> void assertValueInStorage(
             assertThat(storageValue == null ? null : valueTransformer.apply(storageValue), is(expectedValue));
         }
     }
+
+    /**
+     * Asserts data nodes from the distribution zone manager.
+     *
+     * @param distributionZoneManager Distribution zone manager.
+     * @param zoneId Zone id.
+     * @param expectedValue Expected value.
+     * @param timeoutMillis Timeout in milliseconds.
+     * @throws InterruptedException If interrupted.
+     */
+    public static void assertDataNodesFromManager(
+            DistributionZoneManager distributionZoneManager,
+            int zoneId,
+            @Nullable Set<String> expectedValue,
+            long timeoutMillis
+    ) throws InterruptedException {
+        boolean success = waitForCondition(() -> {
+            // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 change this to the causality versioned call to dataNodes.
+            Set<String> dataNodes = distributionZoneManager.dataNodes(zoneId);
+
+            return Objects.equals(dataNodes, expectedValue);
+        }, timeoutMillis);
+
+        // We do a second check simply to print a nice error message in case the condition above is not achieved.

Review Comment:
   Nice))



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();

Review Comment:
   BeforeEach?



-- 
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] alievmirza commented on a diff in pull request #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -175,54 +164,64 @@ void testAlterFilterForDefaultZone() throws Exception {
      *
      * @throws Exception If failed.
      */
-    @Test
-    void testAlterFilterDoNotAffectScaleDown() throws Exception {
-        preparePrerequisites(IMMEDIATE_TIMER_VALUE, 10000);
+    @ParameterizedTest
+    @ValueSource(booleans =  {true, false})

Review Comment:
   done, thanks! 



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();

Review Comment:
   "@BeforeEach?"



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();

Review Comment:
   BeforeEach?



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();
+
+        // Change timers to infinite, add new node, alter filter and check that data nodes was changed.
+        distributionZoneManager.alterZone(
+                ZONE_NAME,
+                new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
+                        .dataNodesAutoAdjustScaleUp(INFINITE_TIMER_VALUE)
+                        .dataNodesAutoAdjustScaleDown(INFINITE_TIMER_VALUE)
+                        .filter(FILTER)
+                        .build()
+        ).get(10_000, TimeUnit.MILLISECONDS);
+
+        topology.putNode(D);
+
+        String newFilter = "$[?(@.region == 'CN')]";

Review Comment:
   Minor-minor: it might be useful to have a comment explaining which nodes match given filer, e.g.
   ```
   // Nodes C and D math the filter.
   String newFilter = "$[?(@.region == 'CN')]";
   ```



-- 
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 #2122: IGNITE-18963 Altering filters must recalculate data nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerAlterFilterTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.distributionzones;
+
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_NAME;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.INFINITE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.assertDataNodesFromManager;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test scenarios when filter of a zone is altered and immediate scale up is triggered.
+ */
+public class DistributionZoneManagerAlterFilterTest  extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME = "zone1";
+
+    private static final int ZONE_ID = 1;
+
+    private static final String FILTER = "$[?(@.storage == 'SSD' || @.region == 'US')]";
+
+    private static final LogicalNode A = new LogicalNode(
+            new ClusterNode("1", "A", new NetworkAddress("localhost", 123)),
+            Map.of("region", "US", "storage", "SSD", "dataRegionSize", "10")
+    );
+
+    private static final LogicalNode B = new LogicalNode(
+            new ClusterNode("2", "B", new NetworkAddress("localhost", 123)),
+            Map.of("region", "EU", "storage", "HHD", "dataRegionSize", "30")
+    );
+
+    private static final LogicalNode C = new LogicalNode(
+            new ClusterNode("3", "C", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "SSD", "dataRegionSize", "20")
+    );
+
+    private static final LogicalNode D = new LogicalNode(
+            new ClusterNode("4", "D", new NetworkAddress("localhost", 123)),
+            Map.of("region", "CN", "storage", "HDD", "dataRegionSize", "20")
+    );
+
+    /**
+     * Tests that node that was added before altering filter is taken into account after altering of a filter and corresponding
+     * immediate scale up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void testAlterFilter() throws Exception {
+        preparePrerequisites();

Review Comment:
   @BeforeEach?



-- 
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