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

[GitHub] [ignite-3] sergeyuttsel opened a new pull request, #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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

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


-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -649,19 +659,8 @@ public Set<String> dataNodes(int zoneId) {
         });
     }
 
-    /**
-     * Asynchronously gets data nodes of the zone using causality token.
-     *
-     * <p>The returned future can be completed with {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
-     * does not exist.
-     *
-     * @param causalityToken Causality token.
-     * @param zoneId Zone id.
-     * @return The future which will be completed with data nodes for the zoneId or with exception.
-     */
-    // TODO: Will be implemented in IGNITE-19506.
-    public CompletableFuture<Set<String>> dataNodes(long causalityToken, int zoneId) {
-        return null;
+    public Set<String> dataNodes(long causalityToken, int zoneId) {

Review Comment:
   I removed this method.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,

Review Comment:
   In this case need to invoke `new CausalityDataNodesEngine(this)` in the DZM constrator or create another solution.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java:
##########
@@ -59,6 +61,18 @@
  */
 @ExtendWith(ConfigurationExtension.class)
 public class BaseDistributionZoneManagerTest extends BaseIgniteAbstractTest {
+    static final String ZONE_NAME_0 = "zone0";
+
+    static final String ZONE_NAME_1 = "zone1";
+
+    static final String ZONE_NAME_2 = "zone2";
+
+    static final LogicalNode NODE_0 = new LogicalNode("node0", "node0", new NetworkAddress("localhost", 123));

Review Comment:
   Fixed



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Asynchronously gets data nodes of the zone using causality token.
+     *
+     * <p>The returned future can be completed with {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        // Get the last scaleUp and scaleDown revisions.
+        if (isZoneRemoved) {
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        // At the dataNodesRevision the zone was created but the data nodes value had not updated yet.
+        // So the data nodes value will be equals to the logical topology on the dataNodesRevision.
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        TreeMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        if (zoneState != null) {
+            subAugmentationMap = new TreeMap<>(zoneState.topologyAugmentationMap()
+                    .subMap(0L, false, causalityToken, true));
+        }
+
+        // Wait if needed when the data nodes value will be updated in the meta storage according to calculated lastScaleUpRevision,
+        // lastScaleDownRevision and causalityToken.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()
+                        && newerCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE
+                        && isScaleUpImmediate) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleDownRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleDown() != newerCfg.getDataNodesAutoAdjustScaleDown()
+                        && newerCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE
+                        && isScaleDownImmediate) {
+                    scaleDownRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleUpRevision == 0 && !olderCfg.getFilter().equals(newerCfg.getFilter())) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+            }
+
+            if ((scaleUpRevision > 0) && (scaleDownRevision > 0)) {
+                break;
+            }
+
+            entryNewerCfg = entryOlderCfg;
+        }
+
+        // The case when there is only one configuration in the history.
+        if (scaleUpRevision == 0 && entryNewerCfg != null) {
+            scaleUpRevision = entryNewerCfg.getKey();
+        }
+
+        if (scaleDownRevision == 0 && entryNewerCfg != null) {
+            scaleDownRevision = entryNewerCfg.getKey();
+        }
+
+        return new IgniteBiTuple<>(scaleUpRevision, scaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest topology event with added nodes and with removed nodes when the zone have
+     * immediate scale up and scale down timers.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownTopologyRevisions(long causalityToken, int zoneId) {
+        Set<NodeWithAttributes> newerLogicalTopology = null;
+
+        long newerTopologyRevision = 0;
+
+        Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), causalityToken);
+
+        long scaleUpTopologyRevision = 0;
+        long scaleDownTopologyRevision = 0;
+
+
+        if (!topologyEntry.empty()) {
+            byte[] newerLogicalTopologyBytes = topologyEntry.value();
+
+            newerLogicalTopology = fromBytes(newerLogicalTopologyBytes);
+
+            newerTopologyRevision = topologyEntry.revision();
+
+            while ((scaleUpTopologyRevision == 0) || (scaleDownTopologyRevision == 0)) {
+                topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), newerTopologyRevision - 1);
+
+                if (!topologyEntry.empty() && newerTopologyRevision == topologyEntry.revision()) {
+                    break;
+                }
+
+                Set<NodeWithAttributes> olderLogicalTopology;
+
+                if (topologyEntry.empty()) {
+                    olderLogicalTopology = emptySet();
+                } else {
+                    byte[] olderLogicalTopologyBytes = topologyEntry.value();
+
+                    olderLogicalTopology = fromBytes(olderLogicalTopologyBytes);
+                }
+
+                Set<NodeWithAttributes> finalNewerLogicalTopology = newerLogicalTopology;
+
+                Set<Node> removedNodes =
+                        olderLogicalTopology.stream()
+                                .filter(node -> !finalNewerLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Set<Node> addedNodes =
+                        newerLogicalTopology.stream()
+                                .filter(node -> !olderLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Map.Entry<Long, ZoneConfiguration> zoneConfigurationEntry = zonesVersionedCfg.get(zoneId)
+                        .floorEntry(newerTopologyRevision);
+
+                if (zoneConfigurationEntry == null) {
+                    break;
+                }
+
+                ZoneConfiguration zoneCfg = zoneConfigurationEntry.getValue();
+
+                if (scaleUpTopologyRevision == 0
+                        && !addedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE) {
+                    scaleUpTopologyRevision = newerTopologyRevision;
+                }
+
+                if (scaleDownTopologyRevision == 0
+                        && !removedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE) {
+                    scaleDownTopologyRevision = newerTopologyRevision;
+                }
+
+                newerLogicalTopology = olderLogicalTopology;
+
+                newerTopologyRevision = topologyEntry.revision();
+
+                if (topologyEntry.empty()) {
+                    break;
+                }
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpTopologyRevision, scaleDownTopologyRevision);
+    }
+
+    /**
+     * Search a value of zoneScaleUpChangeTriggerKey/zoneScaleDownChangeTriggerKey which equals or greater than scaleRevision.
+     * It iterates over the entries in the local metastorage. If there is an entry with a value equal to or greater than
+     * the scaleRevision, then it returns the revision of that entry. If there is no such entry then it returns zero.
+     *
+     * @param scaleRevision Scale revision.
+     * @param zoneId Zone id.
+     * @param triggerKey Trigger key.
+     * @return Future with the revision.
+     */
+    private long searchTriggerKey(Long scaleRevision, int zoneId, ByteArray triggerKey) {
+        System.out.println("waitTriggerKey " + scaleRevision + " " + zoneId);
+
+        System.out.println("waitTriggerKey after registerExactWatch");
+
+        // Must first register the watch listener to listen to new entries. Then read entries from scaleRevision to appliedRevision() + 2.
+        // In this case, we are guaranteed to get all entries from the start revision.
+        long revAfterRegister = msManager.appliedRevision() + 2;
+
+        System.out.println("waitTriggerKey revAfterRegister " + revAfterRegister);
+
+        long upperRevision = max(revAfterRegister, scaleRevision);
+
+        // Gets old entries from storage to check if the expected value was handled before watch listener was registered.
+        List<Entry> entryList = msManager.getLocally(triggerKey.bytes(), scaleRevision, upperRevision);
+
+        long revision = 0;
+
+        for (Entry entry : entryList) {
+            System.out.println("waitTriggerKey iteration revision " + entry.value());
+
+            // scaleRevision is null if the zone was removed.
+            if (scaleRevision == null) {
+                if (entry.value() == null) {
+                    revision = entry.revision();
+
+                    break;
+                }
+            } else {
+                long entryValue = bytesToLong(entry.value());
+
+                if (entryValue >= scaleRevision) {
+                    revision = entry.revision();
+
+                    break;
+                }
+            }
+        }
+
+        return revision;
+    }
+
+    /**
+     * causalityOnUpdateScaleUp.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleUp newScaleUp.
+     */
+    public void causalityOnUpdateScaleUp(long revision, int zoneId, int newScaleUp) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleUp(newScaleUp);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * causalityOnUpdateScaleDown.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleDown newScaleDown.
+     */
+    public void causalityOnUpdateScaleDown(long revision, int zoneId, int newScaleDown) {

Review Comment:
   Fixed. I created separated method for a configuration updating.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,

Review Comment:
   Ok. I used similar approach like on DistributionZoneRebalanceEngine creation.



-- 
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 #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,

Review Comment:
   I don't see any problems to call new CausalityDataNodesEngine(this), we've already have the same logic for `DistributionZoneRebalanceEngine`



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,

Review Comment:
   I don't see any problems to call `new CausalityDataNodesEngine(this)`, we've already have the same logic for `DistributionZoneRebalanceEngine`



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -947,7 +971,8 @@ private void initDataNodesAndTriggerKeysInMetaStorage(
                             revision
                     );
                 } else if (res.getAsBoolean()) {
-                    LOG.debug("Update zones' dataNodes value [zoneId = {}, dataNodes = {}, revision = {}]", zoneId, dataNodes, revision);
+                    LOG.debug("Update zones' dataNodes value [zoneId = {}, dataNodes = {}, revision = {}]",

Review Comment:
   I changed a log level to info in several places where a data nodes value is updated.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -1793,6 +1859,7 @@ public static class ZoneState {
             this.executor = executor;
             topologyAugmentationMap = new ConcurrentSkipListMap<>();
             nodes = emptySet();
+            this.zoneId = zoneId;

Review Comment:
   I will remove this field. It was added for debug logging.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */

Review Comment:
   Thanks. I agree that there are still too few comments and java docs. I will write more.



-- 
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] denis-chudov commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java:
##########
@@ -124,6 +124,7 @@ void setUp() {
         zonesConfiguration = registry.getConfiguration(DistributionZonesConfiguration.KEY);
 
         distributionZoneManager = new DistributionZoneManager(
+                null,

Review Comment:
   It seems weird to me that we have `DistributionZoneCausalityDataNodesTest`, and `DistributionZoneManager#waitZoneProcessing` is never tested - neither there, nor in other places. And this injection of `null` assumes that this method will not work in derived classes. Is it correct that there are no tests for `DistributionZoneManager#waitZoneProcessing`? And, at least, maybe we should care about possible extensions of derived tests and inject the correct value here?



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            // It means that the zone does not exist on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        if (isZoneRemoved) {
+            // It means that the zone was removed on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            // It means that the zone was created but the data nodes value had not updated yet.
+            // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        LOG.info("+++++++ dataNodes zoneState " + zoneState);
+
+        ConcurrentSkipListMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        // The zoneState can be null if the zone was removed.
+        if (zoneState != null) {
+            subAugmentationMap = new ConcurrentSkipListMap<>(zoneState.topologyAugmentationMap()
+                    .headMap(causalityToken, true));
+        }
+
+        // Search the revisions of zoneScaleUpChangeTriggerKey and zoneScaleDownChangeTriggerKey with value greater or equals
+        // to expected one.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        // Choose the highest revision.
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        // Read data nodes value from the meta storage on dataNodesRevision and associated trigger keys.
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        if (dataNodesEntry.value() == null) {
+            LOG.info("+++++++ dataNodes The zone was removed not idempotently");
+            // The zone was removed.
+            // In this case it is impossible to find out the data nodes value idempotently.
+            return emptySet();
+        }
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            // Update the data nodes set with pending data from augmentation map
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        // Apply the filter to get the final data nodes set.
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(

Review Comment:
   Fixed.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java:
##########
@@ -124,6 +124,7 @@ void setUp() {
         zonesConfiguration = registry.getConfiguration(DistributionZonesConfiguration.KEY);
 
         distributionZoneManager = new DistributionZoneManager(
+                null,

Review Comment:
   Fixed. I added a registry instead of null.  `waitZoneProcessing` was removed. Versioned value is used in dataNodes method now, so it is used in many tests. 



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Asynchronously gets data nodes of the zone using causality token.
+     *
+     * <p>The returned future can be completed with {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        // Get the last scaleUp and scaleDown revisions.
+        if (isZoneRemoved) {
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        // At the dataNodesRevision the zone was created but the data nodes value had not updated yet.
+        // So the data nodes value will be equals to the logical topology on the dataNodesRevision.
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        TreeMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        if (zoneState != null) {
+            subAugmentationMap = new TreeMap<>(zoneState.topologyAugmentationMap()
+                    .subMap(0L, false, causalityToken, true));
+        }
+
+        // Wait if needed when the data nodes value will be updated in the meta storage according to calculated lastScaleUpRevision,
+        // lastScaleDownRevision and causalityToken.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()
+                        && newerCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE
+                        && isScaleUpImmediate) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleDownRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleDown() != newerCfg.getDataNodesAutoAdjustScaleDown()
+                        && newerCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE
+                        && isScaleDownImmediate) {
+                    scaleDownRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleUpRevision == 0 && !olderCfg.getFilter().equals(newerCfg.getFilter())) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+            }
+
+            if ((scaleUpRevision > 0) && (scaleDownRevision > 0)) {
+                break;
+            }
+
+            entryNewerCfg = entryOlderCfg;
+        }
+
+        // The case when there is only one configuration in the history.
+        if (scaleUpRevision == 0 && entryNewerCfg != null) {
+            scaleUpRevision = entryNewerCfg.getKey();
+        }
+
+        if (scaleDownRevision == 0 && entryNewerCfg != null) {
+            scaleDownRevision = entryNewerCfg.getKey();
+        }
+
+        return new IgniteBiTuple<>(scaleUpRevision, scaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest topology event with added nodes and with removed nodes when the zone have
+     * immediate scale up and scale down timers.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownTopologyRevisions(long causalityToken, int zoneId) {
+        Set<NodeWithAttributes> newerLogicalTopology = null;
+
+        long newerTopologyRevision = 0;
+
+        Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), causalityToken);
+
+        long scaleUpTopologyRevision = 0;
+        long scaleDownTopologyRevision = 0;
+
+
+        if (!topologyEntry.empty()) {
+            byte[] newerLogicalTopologyBytes = topologyEntry.value();
+
+            newerLogicalTopology = fromBytes(newerLogicalTopologyBytes);
+
+            newerTopologyRevision = topologyEntry.revision();
+
+            while ((scaleUpTopologyRevision == 0) || (scaleDownTopologyRevision == 0)) {
+                topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), newerTopologyRevision - 1);
+
+                if (!topologyEntry.empty() && newerTopologyRevision == topologyEntry.revision()) {
+                    break;
+                }
+
+                Set<NodeWithAttributes> olderLogicalTopology;
+
+                if (topologyEntry.empty()) {
+                    olderLogicalTopology = emptySet();
+                } else {
+                    byte[] olderLogicalTopologyBytes = topologyEntry.value();
+
+                    olderLogicalTopology = fromBytes(olderLogicalTopologyBytes);
+                }
+
+                Set<NodeWithAttributes> finalNewerLogicalTopology = newerLogicalTopology;
+
+                Set<Node> removedNodes =
+                        olderLogicalTopology.stream()
+                                .filter(node -> !finalNewerLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Set<Node> addedNodes =
+                        newerLogicalTopology.stream()
+                                .filter(node -> !olderLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Map.Entry<Long, ZoneConfiguration> zoneConfigurationEntry = zonesVersionedCfg.get(zoneId)
+                        .floorEntry(newerTopologyRevision);
+
+                if (zoneConfigurationEntry == null) {
+                    break;
+                }
+
+                ZoneConfiguration zoneCfg = zoneConfigurationEntry.getValue();
+
+                if (scaleUpTopologyRevision == 0
+                        && !addedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE) {
+                    scaleUpTopologyRevision = newerTopologyRevision;
+                }
+
+                if (scaleDownTopologyRevision == 0
+                        && !removedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE) {
+                    scaleDownTopologyRevision = newerTopologyRevision;
+                }
+
+                newerLogicalTopology = olderLogicalTopology;
+
+                newerTopologyRevision = topologyEntry.revision();
+
+                if (topologyEntry.empty()) {
+                    break;
+                }
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpTopologyRevision, scaleDownTopologyRevision);
+    }
+
+    /**
+     * Search a value of zoneScaleUpChangeTriggerKey/zoneScaleDownChangeTriggerKey which equals or greater than scaleRevision.
+     * It iterates over the entries in the local metastorage. If there is an entry with a value equal to or greater than
+     * the scaleRevision, then it returns the revision of that entry. If there is no such entry then it returns zero.
+     *
+     * @param scaleRevision Scale revision.
+     * @param zoneId Zone id.
+     * @param triggerKey Trigger key.
+     * @return Future with the revision.
+     */
+    private long searchTriggerKey(Long scaleRevision, int zoneId, ByteArray triggerKey) {
+        System.out.println("waitTriggerKey " + scaleRevision + " " + zoneId);
+
+        System.out.println("waitTriggerKey after registerExactWatch");
+
+        // Must first register the watch listener to listen to new entries. Then read entries from scaleRevision to appliedRevision() + 2.
+        // In this case, we are guaranteed to get all entries from the start revision.
+        long revAfterRegister = msManager.appliedRevision() + 2;
+
+        System.out.println("waitTriggerKey revAfterRegister " + revAfterRegister);
+
+        long upperRevision = max(revAfterRegister, scaleRevision);
+
+        // Gets old entries from storage to check if the expected value was handled before watch listener was registered.
+        List<Entry> entryList = msManager.getLocally(triggerKey.bytes(), scaleRevision, upperRevision);

Review Comment:
   Maybe yes, maybe no :) I had created https://issues.apache.org/jira/browse/IGNITE-19735 to move all *Locally methods to another interface. That is why this method is deprecated.



-- 
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 #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneCausalityDataNodesTest.java:
##########
@@ -0,0 +1,683 @@
+/*
+ * 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 java.util.Collections.emptySet;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.cluster.management.topology.LogicalTopologyImpl.LOGICAL_TOPOLOGY_KEY;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneLogicalTopologyPrefix;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesDataNodesPrefix;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyVersionKey;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.IgniteUtils.startsWith;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologySnapshot;
+import org.apache.ignite.internal.distributionzones.DistributionZoneConfigurationParameters.Builder;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.exception.DistributionZoneNotFoundException;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.EntryEvent;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for causality data nodes updating in {@link DistributionZoneManager}.
+ */
+public class DistributionZoneCausalityDataNodesTest extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME_1 = "zone1";
+
+    private static final String ZONE_NAME_2 = "zone2";
+
+    private static final int ZONE_ID_1 = 1;
+
+    private static final int ZONE_ID_2 = 2;
+
+    private static final LogicalNode NODE_0 =
+            new LogicalNode("node_id_0", "node_name_0", new NetworkAddress("localhost", 123));
+
+    private static final LogicalNode NODE_1 =
+            new LogicalNode("node_id_1", "node_name_1", new NetworkAddress("localhost", 123));
+
+    private static final LogicalNode NODE_2 =
+            new LogicalNode("node_id_2", "node_name_2", new NetworkAddress("localhost", 123));
+
+    /**
+     * Contains futures that is completed when the topology watch listener receive the event with expected data nodes.

Review Comment:
   >with expected data nodes
   
   I would say not data nodes, but logical topology
   



##########
.idea/checkstyle-idea.xml:
##########
@@ -1,27 +1,16 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <project version="4">
-  <component name="CheckStyle-IDEA" serialisationVersion="2">
-    <checkstyleVersion>9.3</checkstyleVersion>
-    <scanScope>JavaOnly</scanScope>
-    <option name="thirdPartyClasspath" />
-    <option name="activeLocationIds">
-      <option value="610eb513-51a7-4863-aaf8-feef91fd2055" />
-    </option>
-    <option name="locations">
-      <list>
-        <ConfigurationLocation id="bundled-sun-checks" type="BUNDLED" scope="All" description="Sun Checks">(bundled)</ConfigurationLocation>
-        <ConfigurationLocation id="bundled-google-checks" type="BUNDLED" scope="All" description="Google Checks">(bundled)</ConfigurationLocation>
-        <ConfigurationLocation id="610eb513-51a7-4863-aaf8-feef91fd2055" type="LOCAL_FILE" scope="All" description="Ignite3">
-          $PROJECT_DIR$/check-rules/checkstyle-rules.xml
-          <option name="properties">
-            <map>
-              <entry key="checkstyle.header.file" value="" />
-              <entry key="org.checkstyle.google.suppressionfilter.config" value="$PROJECT_DIR$/check-rules/checkstyle-suppressions.xml" />
-              <entry key="org.checkstyle.google.suppressionxpathfilter.config" value="" />
-            </map>
-          </option>
-        </ConfigurationLocation>
-      </list>
+  <component name="CheckStyle-IDEA">

Review Comment:
   ?



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneCausalityDataNodesTest.java:
##########
@@ -0,0 +1,683 @@
+/*
+ * 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 java.util.Collections.emptySet;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.cluster.management.topology.LogicalTopologyImpl.LOGICAL_TOPOLOGY_KEY;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneLogicalTopologyPrefix;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesDataNodesPrefix;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyVersionKey;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.IgniteUtils.startsWith;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologySnapshot;
+import org.apache.ignite.internal.distributionzones.DistributionZoneConfigurationParameters.Builder;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.exception.DistributionZoneNotFoundException;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.EntryEvent;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for causality data nodes updating in {@link DistributionZoneManager}.
+ */
+public class DistributionZoneCausalityDataNodesTest extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME_1 = "zone1";
+
+    private static final String ZONE_NAME_2 = "zone2";
+
+    private static final int ZONE_ID_1 = 1;
+
+    private static final int ZONE_ID_2 = 2;
+
+    private static final LogicalNode NODE_0 =
+            new LogicalNode("node_id_0", "node_name_0", new NetworkAddress("localhost", 123));
+
+    private static final LogicalNode NODE_1 =
+            new LogicalNode("node_id_1", "node_name_1", new NetworkAddress("localhost", 123));
+
+    private static final LogicalNode NODE_2 =
+            new LogicalNode("node_id_2", "node_name_2", new NetworkAddress("localhost", 123));
+
+    /**
+     * Contains futures that is completed when the topology watch listener receive the event with expected data nodes.
+     * Mapping of node names -> future with event revision.
+     */
+    private final ConcurrentHashMap<Set<String>, CompletableFuture<Long>> topologyRevisions = new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the data nodes watch listener receive the event with expected zone id and data nodes.
+     * Mapping of zone id and node names -> future with event revision.
+     */
+    private final ConcurrentHashMap<IgniteBiTuple<Integer, Set<String>>, CompletableFuture<Long>> zoneDataNodesRevisions =
+            new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the scale up update listener receive the event with expected zone id.
+     * Mapping of zone id -> future with event revision.
+     */
+    private final ConcurrentHashMap<Integer, CompletableFuture<Long>> zoneScaleUpRevisions = new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the scale down update listener receive the event with expected zone id.
+     * Mapping of zone id -> future with event revision.
+     */
+    private final ConcurrentHashMap<Integer, CompletableFuture<Long>> zoneScaleDownRevisions = new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the zone configuration listener receive the zone creation event with expected zone id.
+     * Mapping of zone id -> future with event revision.
+     */
+    private final ConcurrentHashMap<Integer, CompletableFuture<Long>> createZoneRevisions = new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the zone configuration listener receive the zone dropping event with expected zone id.
+     * Mapping of zone id -> future with event revision.
+     */
+    private final ConcurrentHashMap<Integer, CompletableFuture<Long>> dropZoneRevisions = new ConcurrentHashMap<>();
+
+    @BeforeEach
+    void beforeEach() throws NodeStoppingException {
+        metaStorageManager.registerPrefixWatch(zoneLogicalTopologyPrefix(), createMetastorageTopologyListener());
+
+        metaStorageManager.registerPrefixWatch(zonesDataNodesPrefix(), createMetastorageDataNodesListener());
+
+        ZonesConfigurationListener zonesConfigurationListener = new ZonesConfigurationListener();
+
+        zonesConfiguration.distributionZones().listenElements(zonesConfigurationListener);
+        zonesConfiguration.distributionZones().any().dataNodesAutoAdjustScaleUp().listen(onUpdateScaleUp());
+        zonesConfiguration.distributionZones().any().dataNodesAutoAdjustScaleDown().listen(onUpdateScaleDown());
+
+        zonesConfiguration.defaultDistributionZone().listen(zonesConfigurationListener);
+        zonesConfiguration.defaultDistributionZone().dataNodesAutoAdjustScaleUp().listen(onUpdateScaleUp());
+        zonesConfiguration.defaultDistributionZone().dataNodesAutoAdjustScaleDown().listen(onUpdateScaleDown());
+
+        distributionZoneManager.start();
+
+        metaStorageManager.deployWatches();
+    }
+
+    /**
+     * Tests data nodes updating on a topology leap.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void topologyLeapUpdate() throws Exception {
+        // Prerequisite.
+
+        // Create the zone with immediate timers.
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_1)
+                                .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                                .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Create the zone with not immediate timers.
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_2)
+                                .dataNodesAutoAdjustScaleUp(1)
+                                .dataNodesAutoAdjustScaleDown(1)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Create logical topology with NODE_0 and NODE_1.
+        topology.putNode(NODE_0);
+
+        Set<LogicalNode> twoNodes1 = Set.of(NODE_0, NODE_1);
+        Set<String> twoNodesNames1 = Set.of(NODE_0.name(), NODE_1.name());
+
+        CompletableFuture<Long> dataNodesUpdateRevision = getZoneDataNodesRevision(ZONE_ID_2, twoNodes1);
+
+        // Check that data nodes value of both zone is NODE_0 and NODE_1.
+        long topologyRevision1 = putNodeInLogicalTopologyAndGetRevision(NODE_1, twoNodes1);
+
+        CompletableFuture<Set<String>> dataNodesFut0 = distributionZoneManager.dataNodes(topologyRevision1, ZONE_ID_1);
+        assertThat(dataNodesFut0, willBe(twoNodesNames1));
+
+        long dataNodesRevisionZone = dataNodesUpdateRevision.get(3, SECONDS);
+
+        CompletableFuture<Set<String>> dataNodesFut1 = distributionZoneManager.dataNodes(dataNodesRevisionZone, ZONE_ID_2);
+        assertThat(dataNodesFut1, willBe(twoNodesNames1));
+
+        // Test steps.
+
+        // Change logical topology. NODE_1 is left. NODE_2 is added.
+        Set<LogicalNode> twoNodes2 = Set.of(NODE_0, NODE_2);
+        Set<String> twoNodesNames2 = Set.of(NODE_0.name(), NODE_2.name());
+
+        dataNodesUpdateRevision = getZoneDataNodesRevision(ZONE_ID_2, twoNodes2);
+
+        long topologyRevision2 = fireTopologyLeapAndGetRevision(twoNodes2);
+
+        // Check that data nodes value of the zone with immediate timers with the topology update revision is NODE_0 and NODE_2.
+        CompletableFuture<Set<String>> dataNodesFut3 = distributionZoneManager.dataNodes(topologyRevision2, ZONE_ID_1);
+        assertThat(dataNodesFut3, willBe(twoNodesNames2));
+
+        // Check that data nodes value of the zone with not immediate timers with the topology update revision is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut4 = distributionZoneManager.dataNodes(topologyRevision2, ZONE_ID_2);
+        assertThat(dataNodesFut4, willBe(twoNodesNames1));
+
+        // Check that data nodes value of the zone with not immediate timers with the data nodes update revision is NODE_0 and NODE_2.
+        dataNodesRevisionZone = dataNodesUpdateRevision.get(3, SECONDS);
+        CompletableFuture<Set<String>> dataNodesFut5 = distributionZoneManager.dataNodes(dataNodesRevisionZone, ZONE_ID_2);
+        assertThat(dataNodesFut5, willBe(twoNodesNames2));
+    }
+
+    /**
+     * Tests data nodes updating on a scale up changing.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void dataNodesUpdatedAfterScaleUpChanged() throws Exception {
+        // Prerequisite.
+
+        // Create the zone with immediate scale up timer and immediate scale down timer.
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_1)
+                                .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                                .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Create logical topology with NODE_0.
+        Set<LogicalNode> oneNode = Set.of(NODE_0);
+        Set<String> oneNodeName = Set.of(NODE_0.name());
+
+        long topologyRevision1 = putNodeInLogicalTopologyAndGetRevision(NODE_1, oneNode);
+
+        // Check that data nodes value of the the zone is NODE_0.
+        CompletableFuture<Set<String>> dataNodesFut1 = distributionZoneManager.dataNodes(topologyRevision1, ZONE_ID_1);
+        assertThat(dataNodesFut1, willBe(oneNodeName));
+
+        // Changes a scale up timer to not immediate.
+        distributionZoneManager.alterZone(
+                        ZONE_NAME_1,
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_1)
+                                .dataNodesAutoAdjustScaleUp(10000)
+                                .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Test steps.
+
+        // Change logical topology. NODE_1 is added.
+        Set<LogicalNode> twoNodes = Set.of(NODE_0, NODE_1);
+        Set<String> twoNodesNames = Set.of(NODE_0.name(), NODE_1.name());
+
+        long topologyRevision2 = putNodeInLogicalTopologyAndGetRevision(NODE_1, twoNodes);
+
+        // Check that data nodes value of the zone with the topology update revision is NODE_0 because scale up timer has not fired yet.
+        CompletableFuture<Set<String>> dataNodesFut2 = distributionZoneManager.dataNodes(topologyRevision2, ZONE_ID_1);
+        assertThat(dataNodesFut2, willBe(oneNode));
+
+        // Change scale up value to immediate.
+        long scaleUpRevision = alterZoneScaleUpAndGetRevision(ZONE_NAME_1, IMMEDIATE_TIMER_VALUE);
+
+        // Check that data nodes value of the zone with the scale up update revision is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut3 = distributionZoneManager.dataNodes(scaleUpRevision, ZONE_ID_1);
+        assertThat(dataNodesFut3, willBe(twoNodesNames));
+    }
+
+    /**
+     * Tests data nodes updating on a scale down changing.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void dataNodesUpdatedAfterScaleDownChanged() throws Exception {
+        // Prerequisite.
+
+        // Create the zone with immediate scale up timer and not immediate scale down timer.
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_1)
+                                .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                                .dataNodesAutoAdjustScaleDown(10000)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Create logical topology with NODE_0 and NODE_1.
+        topology.putNode(NODE_0);
+
+        Set<LogicalNode> twoNodes = Set.of(NODE_0, NODE_1);
+        Set<String> twoNodesNames = Set.of(NODE_0.name(), NODE_1.name());
+
+        long topologyRevision1 = putNodeInLogicalTopologyAndGetRevision(NODE_1, twoNodes);
+
+        // Check that data nodes value of the the zone is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut1 = distributionZoneManager.dataNodes(topologyRevision1, ZONE_ID_1);
+        assertThat(dataNodesFut1, willBe(twoNodesNames));
+
+        // Test steps.
+
+        // Change logical topology. NODE_1 is added.
+        Set<LogicalNode> oneNode = Set.of(NODE_0);
+        Set<String> oneNodeName = Set.of(NODE_0.name());
+
+        long topologyRevision2 = removeNodeInLogicalTopologyAndGetRevision(Set.of(NODE_1), oneNode);
+
+        // Check that data nodes value of the zone with the topology update revision is NODE_0 because scale down timer has not fired yet.
+        CompletableFuture<Set<String>> dataNodesFut2 = distributionZoneManager.dataNodes(topologyRevision2, ZONE_ID_1);
+        assertThat(dataNodesFut2, willBe(twoNodesNames));
+
+        // Change scale down value to immediate.
+        long scaleDownRevision = alterZoneScaleDownAndGetRevision(ZONE_NAME_1, IMMEDIATE_TIMER_VALUE);
+
+        // Check that data nodes value of the zone with the scale down update revision is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut3 = distributionZoneManager.dataNodes(scaleDownRevision, ZONE_ID_1);
+        assertThat(dataNodesFut3, willBe(oneNodeName));
+    }
+
+    /**
+     * Tests data nodes obtaining with revision before a zone creation and after a zone dropping.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void createThenDropZone() throws Exception {
+        // Prerequisite.
+
+        // Create logical topology with NODE_0 and NODE_1.
+        topology.putNode(NODE_0);
+        topology.putNode(NODE_1);
+
+        Set<String> twoNodesNames = Set.of(NODE_0.name(), NODE_1.name());
+
+        // Test steps.
+
+        // Create a zone.
+        long createZoneRevision = createZoneAndGetRevision(ZONE_NAME_1, ZONE_ID_1, IMMEDIATE_TIMER_VALUE, IMMEDIATE_TIMER_VALUE);
+
+        // Check that data nodes value of the zone with the revision lower than the create zone revision is absent.
+        CompletableFuture<Set<String>> dataNodesFut1 = distributionZoneManager.dataNodes(createZoneRevision - 1, ZONE_ID_1);
+        assertThrows(DistributionZoneNotFoundException.class, () -> dataNodesFut1.get(3, SECONDS));
+
+        // Check that data nodes value of the zone with the create zone revision is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut2 = distributionZoneManager.dataNodes(createZoneRevision, ZONE_ID_1);
+        assertThat(dataNodesFut2, willBe(twoNodesNames));
+
+        // Drop the zone.
+        long dropZoneRevision = dropZoneAndGetRevision(ZONE_NAME_1);
+
+        // Check that data nodes value of the zone with the drop zone revision is absent.
+        CompletableFuture<Set<String>> dataNodesFut3 = distributionZoneManager.dataNodes(dropZoneRevision, ZONE_ID_1);
+        assertThrows(DistributionZoneNotFoundException.class, () -> dataNodesFut3.get(3, SECONDS));
+    }
+
+    /**
+     * Puts a given node as a part of the logical topology and return revision of a topology watch listener event.
+     *
+     * @param node Node to put.
+     * @param expectedTopology Expected topology for future completing.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long putNodeInLogicalTopologyAndGetRevision(
+            LogicalNode node,
+            Set<LogicalNode> expectedTopology
+    ) throws Exception {
+        Set<String> nodeNames = expectedTopology.stream().map(ClusterNode::name).collect(toSet());
+
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        topologyRevisions.put(nodeNames, revisionFut);
+
+        topology.putNode(node);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Removes given nodes from the logical topology and return revision of a topology watch listener event.
+     *
+     * @param nodes Nodes to remove.
+     * @param expectedTopology Expected topology for future completing.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long removeNodeInLogicalTopologyAndGetRevision(
+            Set<LogicalNode> nodes,
+            Set<LogicalNode> expectedTopology
+    ) throws Exception {
+        Set<String> nodeNames = expectedTopology.stream().map(ClusterNode::name).collect(toSet());
+
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        topologyRevisions.put(nodeNames, revisionFut);
+
+        topology.removeNodes(nodes);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Changes data nodes in logical topology and return revision of a topology watch listener event.
+     *
+     * @param nodes Nodes to remove.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long fireTopologyLeapAndGetRevision(Set<LogicalNode> nodes) throws Exception {
+        Set<String> nodeNames = nodes.stream().map(ClusterNode::name).collect(toSet());
+
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        topologyRevisions.put(nodeNames, revisionFut);
+
+        long topVer = topology.getLogicalTopology().version() + 1;
+
+        clusterStateStorage.put(LOGICAL_TOPOLOGY_KEY, ByteUtils.toBytes(new LogicalTopologySnapshot(topVer, nodes)));
+
+        topology.fireTopologyLeap();
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Changes a scale up timer value of a zone and return the revision of a zone update event.
+     *
+     * @param zoneName Zone name.
+     * @param scaleUp New scale up value.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long alterZoneScaleUpAndGetRevision(String zoneName, int scaleUp) throws Exception {
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        int zoneId = distributionZoneManager.getZoneId(zoneName);
+
+        zoneScaleUpRevisions.put(zoneId, revisionFut);
+
+        distributionZoneManager.alterZone(zoneName, new Builder(zoneName)
+                        .dataNodesAutoAdjustScaleUp(scaleUp).build())
+                .get(3, SECONDS);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Changes a scale down timer value of a zone and return the revision of a zone update event.
+     *
+     * @param zoneName Zone name.
+     * @param scaleDown New scale down value.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long alterZoneScaleDownAndGetRevision(String zoneName, int scaleDown) throws Exception {
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        int zoneId = distributionZoneManager.getZoneId(zoneName);
+
+        zoneScaleDownRevisions.put(zoneId, revisionFut);
+
+        distributionZoneManager.alterZone(zoneName, new Builder(zoneName)
+                        .dataNodesAutoAdjustScaleDown(scaleDown).build())
+                .get(3, SECONDS);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Creates a zone and return the revision of a create zone event.
+     *
+     * @param zoneName Zone name.
+     * @param zoneId Zone id.
+     * @param scaleUp Scale up value.
+     * @param scaleDown Scale down value.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long createZoneAndGetRevision(String zoneName, int zoneId, int scaleUp, int scaleDown) throws Exception {
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        createZoneRevisions.put(zoneId, revisionFut);
+
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(zoneName)
+                                .dataNodesAutoAdjustScaleUp(scaleUp)
+                                .dataNodesAutoAdjustScaleDown(scaleDown)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Drops a zone and return the revision of a drop zone event.
+     *
+     * @param zoneName Zone name.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long dropZoneAndGetRevision(String zoneName) throws Exception {
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        int zoneId = distributionZoneManager.getZoneId(zoneName);
+
+        dropZoneRevisions.put(zoneId, revisionFut);
+
+        distributionZoneManager.dropZone(zoneName).get(3, SECONDS);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Removes given nodes from the logical topology and return future with revision of topology watch listener event.
+     *
+     * @param zoneId Zone id.
+     * @param nodes Expected data nodes.
+     * @return Future with revision.
+     */
+    private CompletableFuture<Long> getZoneDataNodesRevision(int zoneId, Set<LogicalNode> nodes) {

Review Comment:
   Something wrong with the name or with javadoc



-- 
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 #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java:
##########
@@ -59,6 +61,18 @@
  */
 @ExtendWith(ConfigurationExtension.class)
 public class BaseDistributionZoneManagerTest extends BaseIgniteAbstractTest {
+    static final String ZONE_NAME_0 = "zone0";
+
+    static final String ZONE_NAME_1 = "zone1";
+
+    static final String ZONE_NAME_2 = "zone2";
+
+    static final LogicalNode NODE_0 = new LogicalNode("node0", "node0", new NetworkAddress("localhost", 123));

Review Comment:
   It's better to use different node name and id, just in case.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -341,6 +341,7 @@ private PartialNode startPartialNode(
         SchemaManager schemaManager = new SchemaManager(registry, tablesConfig, metaStorageMgr);
 
         DistributionZoneManager distributionZoneManager = new DistributionZoneManager(
+                null,

Review Comment:
   I have added a registry



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;

Review Comment:
   Ticket is created https://issues.apache.org/jira/browse/IGNITE-20050



-- 
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] denis-chudov commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -465,23 +468,30 @@ public static <T> void assertValueInStorage(
      */
     public static void assertDataNodesFromManager(
             DistributionZoneManager distributionZoneManager,
+            Supplier<Long> causalityToken,
             int zoneId,
             @Nullable Set<LogicalNode> expectedValue,
             long timeoutMillis
-    ) throws InterruptedException {
+    ) throws InterruptedException, ExecutionException, TimeoutException {
         Set<String> expectedValueNames =
                 expectedValue == null ? null : expectedValue.stream().map(ClusterNode::name).collect(Collectors.toSet());
 
         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);
+            Set<String> dataNodes = null;
+            try {
+                dataNodes = distributionZoneManager.dataNodes(causalityToken.get(), zoneId).get(5, TimeUnit.SECONDS);
+            } catch (Exception e) {
+                // Ignore
+            }
 
             return Objects.equals(dataNodes, expectedValueNames);
         }, timeoutMillis);
 
         // We do a second check simply to print a nice error message in case the condition above is not achieved.
         if (!success) {
-            Set<String> dataNodes = distributionZoneManager.dataNodes(zoneId);
+            Set<String> dataNodes = null;
+
+            dataNodes = distributionZoneManager.dataNodes(causalityToken.get(), zoneId).get(5, TimeUnit.SECONDS);

Review Comment:
   can be merged into one line



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -598,69 +598,69 @@ private CompletableFuture<?> onTableCreate(ConfigurationNotificationEvent<TableV
             CatalogTableDescriptor tableDescriptor = toTableDescriptor(ctx.newValue());
             CatalogZoneDescriptor zoneDescriptor = getZoneDescriptor(tableDescriptor.zoneId());
 
-            List<Set<Assignment>> assignments;
+            CompletableFuture<List<Set<Assignment>>> assignmentsFuture;
 
             int tableId = tableDescriptor.id();
 
             // Check if the table already has assignments in the vault.
             // So, it means, that it is a recovery process and we should use the vault assignments instead of calculation for the new ones.
             if (partitionAssignments(vaultManager, tableId, 0) != null) {
-                assignments = tableAssignments(vaultManager, tableId, zoneDescriptor.partitions());
+                assignmentsFuture = completedFuture(tableAssignments(vaultManager, tableId, zoneDescriptor.partitions()));
             } else {
-                assignments = AffinityUtils.calculateAssignments(
-                        // TODO: https://issues.apache.org/jira/browse/IGNITE-19425 use data nodes from DistributionZoneManager instead.
-                        baselineMgr.nodes().stream().map(ClusterNode::name).collect(toList()),
-                        zoneDescriptor.partitions(),
-                        zoneDescriptor.replicas()
-                );
+                assignmentsFuture = distributionZoneManager.dataNodes(ctx.storageRevision(), tableDescriptor.zoneId())
+                        .thenApply(dataNodes -> AffinityUtils.calculateAssignments(
+                                dataNodes,
+                                zoneDescriptor.partitions(),
+                                zoneDescriptor.replicas()
+                        ));
             }
 
-            assert !assignments.isEmpty() : "Couldn't create the table with empty assignments.";
-
             CompletableFuture<?> createTableFut = createTableLocally(
                     ctx.storageRevision(),
                     tableDescriptor,
                     zoneDescriptor,
-                    assignments
+                    assignmentsFuture
             ).whenComplete((v, e) -> {
                 if (e == null) {
                     for (var listener : assignmentsChangeListeners) {
                         listener.accept(this);
                     }
                 }
-            });
-
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 Probably should be reworked so that
-            // the future is returned along with createTableFut. Right now it will break some tests.
-            writeTableAssignmentsToMetastore(tableId, assignments);
+            }).thenCompose(ignored -> writeTableAssignmentsToMetastore(tableId, assignmentsFuture));
 
             return createTableFut;
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    private CompletableFuture<Boolean> writeTableAssignmentsToMetastore(int tableId, List<Set<Assignment>> assignments) {
-        assert !assignments.isEmpty();
+    private CompletableFuture<Boolean> writeTableAssignmentsToMetastore(
+            int tableId,
+            CompletableFuture<List<Set<Assignment>>> assignmentsFuture
+    ) {
+        return assignmentsFuture.thenCompose(newAssignments -> {
+            assert !newAssignments.isEmpty();
 
-        List<Operation> partitionAssignments = new ArrayList<>(assignments.size());
+            List<Operation> partitionAssignments = new ArrayList<>(newAssignments.size());
 
-        for (int i = 0; i < assignments.size(); i++) {
-            partitionAssignments.add(put(
-                    stablePartAssignmentsKey(
-                            new TablePartitionId(tableId, i)),
-                    ByteUtils.toBytes(assignments.get(i))));
-        }
+            for (int i = 0; i < newAssignments.size(); i++) {
+                partitionAssignments.add(put(
+                        stablePartAssignmentsKey(
+                                new TablePartitionId(tableId, i)),
+                        ByteUtils.toBytes(newAssignments.get(i))));
+            }
 
-        Condition condition = Conditions.notExists(new ByteArray(partitionAssignments.get(0).key()));
+            Condition condition = Conditions.notExists(new ByteArray(partitionAssignments.get(0).key()));
 
-        return metaStorageMgr
-                .invoke(condition, partitionAssignments, Collections.emptyList())
-                .exceptionally(e -> {
-                    LOG.error("Couldn't write assignments to metastore", e);
+            return metaStorageMgr
+                    .invoke(condition, partitionAssignments, Collections.emptyList())
+                    .exceptionally(e -> {
+                        LOG.error("Couldn't write assignments to metastore", e);
+
+                        return null;
+                    });
+        });
 

Review Comment:
   pls remove empty line



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -793,6 +792,10 @@ public Set<String> dataNodes(int zoneId) {
         });
     }
 
+    public CompletableFuture<Set<String>> versionedDataNodes(int zoneId, long revision) {

Review Comment:
   - I'd rather user `dataNodes` instead of `versionedDataNodes`
   - It's not revision it's causalityToken.
   - causalityToken should be the first parameter, see `org.apache.ignite.internal.table.distributed.TableManager#tableAsync(long, java.util.UUID)` or `org.apache.ignite.internal.index.IndexManager#createIndexLocally` for example.
   - Where's the javadoc?



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -793,6 +792,10 @@ public Set<String> dataNodes(int zoneId) {
         });
     }
 
+    public CompletableFuture<Set<String>> versionedDataNodes(int zoneId, long revision) {

Review Comment:
   Fixed. I think that the java doc will be changed with more details.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/testFixtures/java/org/apache/ignite/internal/distributionzones/DistributionZonesTestUtil.java:
##########
@@ -465,23 +468,30 @@ public static <T> void assertValueInStorage(
      */
     public static void assertDataNodesFromManager(
             DistributionZoneManager distributionZoneManager,
+            Supplier<Long> causalityToken,
             int zoneId,
             @Nullable Set<LogicalNode> expectedValue,
             long timeoutMillis
-    ) throws InterruptedException {
+    ) throws InterruptedException, ExecutionException, TimeoutException {
         Set<String> expectedValueNames =
                 expectedValue == null ? null : expectedValue.stream().map(ClusterNode::name).collect(Collectors.toSet());
 
         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);
+            Set<String> dataNodes = null;
+            try {
+                dataNodes = distributionZoneManager.dataNodes(causalityToken.get(), zoneId).get(5, TimeUnit.SECONDS);
+            } catch (Exception e) {
+                // Ignore
+            }
 
             return Objects.equals(dataNodes, expectedValueNames);
         }, timeoutMillis);
 
         // We do a second check simply to print a nice error message in case the condition above is not achieved.
         if (!success) {
-            Set<String> dataNodes = distributionZoneManager.dataNodes(zoneId);
+            Set<String> dataNodes = null;
+
+            dataNodes = distributionZoneManager.dataNodes(causalityToken.get(), zoneId).get(5, TimeUnit.SECONDS);

Review Comment:
   Fixed



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -598,69 +598,69 @@ private CompletableFuture<?> onTableCreate(ConfigurationNotificationEvent<TableV
             CatalogTableDescriptor tableDescriptor = toTableDescriptor(ctx.newValue());
             CatalogZoneDescriptor zoneDescriptor = getZoneDescriptor(tableDescriptor.zoneId());
 
-            List<Set<Assignment>> assignments;
+            CompletableFuture<List<Set<Assignment>>> assignmentsFuture;
 
             int tableId = tableDescriptor.id();
 
             // Check if the table already has assignments in the vault.
             // So, it means, that it is a recovery process and we should use the vault assignments instead of calculation for the new ones.
             if (partitionAssignments(vaultManager, tableId, 0) != null) {
-                assignments = tableAssignments(vaultManager, tableId, zoneDescriptor.partitions());
+                assignmentsFuture = completedFuture(tableAssignments(vaultManager, tableId, zoneDescriptor.partitions()));
             } else {
-                assignments = AffinityUtils.calculateAssignments(
-                        // TODO: https://issues.apache.org/jira/browse/IGNITE-19425 use data nodes from DistributionZoneManager instead.
-                        baselineMgr.nodes().stream().map(ClusterNode::name).collect(toList()),
-                        zoneDescriptor.partitions(),
-                        zoneDescriptor.replicas()
-                );
+                assignmentsFuture = distributionZoneManager.dataNodes(ctx.storageRevision(), tableDescriptor.zoneId())
+                        .thenApply(dataNodes -> AffinityUtils.calculateAssignments(
+                                dataNodes,
+                                zoneDescriptor.partitions(),
+                                zoneDescriptor.replicas()
+                        ));
             }
 
-            assert !assignments.isEmpty() : "Couldn't create the table with empty assignments.";
-
             CompletableFuture<?> createTableFut = createTableLocally(
                     ctx.storageRevision(),
                     tableDescriptor,
                     zoneDescriptor,
-                    assignments
+                    assignmentsFuture
             ).whenComplete((v, e) -> {
                 if (e == null) {
                     for (var listener : assignmentsChangeListeners) {
                         listener.accept(this);
                     }
                 }
-            });
-
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19506 Probably should be reworked so that
-            // the future is returned along with createTableFut. Right now it will break some tests.
-            writeTableAssignmentsToMetastore(tableId, assignments);
+            }).thenCompose(ignored -> writeTableAssignmentsToMetastore(tableId, assignmentsFuture));
 
             return createTableFut;
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    private CompletableFuture<Boolean> writeTableAssignmentsToMetastore(int tableId, List<Set<Assignment>> assignments) {
-        assert !assignments.isEmpty();
+    private CompletableFuture<Boolean> writeTableAssignmentsToMetastore(
+            int tableId,
+            CompletableFuture<List<Set<Assignment>>> assignmentsFuture
+    ) {
+        return assignmentsFuture.thenCompose(newAssignments -> {
+            assert !newAssignments.isEmpty();
 
-        List<Operation> partitionAssignments = new ArrayList<>(assignments.size());
+            List<Operation> partitionAssignments = new ArrayList<>(newAssignments.size());
 
-        for (int i = 0; i < assignments.size(); i++) {
-            partitionAssignments.add(put(
-                    stablePartAssignmentsKey(
-                            new TablePartitionId(tableId, i)),
-                    ByteUtils.toBytes(assignments.get(i))));
-        }
+            for (int i = 0; i < newAssignments.size(); i++) {
+                partitionAssignments.add(put(
+                        stablePartAssignmentsKey(
+                                new TablePartitionId(tableId, i)),
+                        ByteUtils.toBytes(newAssignments.get(i))));
+            }
 
-        Condition condition = Conditions.notExists(new ByteArray(partitionAssignments.get(0).key()));
+            Condition condition = Conditions.notExists(new ByteArray(partitionAssignments.get(0).key()));
 
-        return metaStorageMgr
-                .invoke(condition, partitionAssignments, Collections.emptyList())
-                .exceptionally(e -> {
-                    LOG.error("Couldn't write assignments to metastore", e);
+            return metaStorageMgr
+                    .invoke(condition, partitionAssignments, Collections.emptyList())
+                    .exceptionally(e -> {
+                        LOG.error("Couldn't write assignments to metastore", e);
+
+                        return null;
+                    });
+        });
 

Review Comment:
   Fixed



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

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

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


[GitHub] [ignite-3] denis-chudov commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */

Review Comment:
   As someone who did not participate in the development, I would like to see some more elaborated comment, what does this engine serves for, why it is called "engine" and "manager" at the same time, what are causality data nodes, etc.



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -1793,6 +1859,7 @@ public static class ZoneState {
             this.executor = executor;
             topologyAugmentationMap = new ConcurrentSkipListMap<>();
             nodes = emptySet();
+            this.zoneId = zoneId;

Review Comment:
   you forgot to add zoneId to parameters



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;

Review Comment:
   Is this map cleaned up somewhere? And the corresponding vault entries as well



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -649,19 +659,8 @@ public Set<String> dataNodes(int zoneId) {
         });
     }
 
-    /**
-     * Asynchronously gets data nodes of the zone using causality token.
-     *
-     * <p>The returned future can be completed with {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
-     * does not exist.
-     *
-     * @param causalityToken Causality token.
-     * @param zoneId Zone id.
-     * @return The future which will be completed with data nodes for the zoneId or with exception.
-     */
-    // TODO: Will be implemented in IGNITE-19506.
-    public CompletableFuture<Set<String>> dataNodes(long causalityToken, int zoneId) {
-        return null;
+    public Set<String> dataNodes(long causalityToken, int zoneId) {

Review Comment:
   is another method without token supposed to be removed? I dont see any usages except of tests



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Asynchronously gets data nodes of the zone using causality token.
+     *
+     * <p>The returned future can be completed with {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        // Get the last scaleUp and scaleDown revisions.
+        if (isZoneRemoved) {
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        // At the dataNodesRevision the zone was created but the data nodes value had not updated yet.
+        // So the data nodes value will be equals to the logical topology on the dataNodesRevision.
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        TreeMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        if (zoneState != null) {
+            subAugmentationMap = new TreeMap<>(zoneState.topologyAugmentationMap()
+                    .subMap(0L, false, causalityToken, true));
+        }
+
+        // Wait if needed when the data nodes value will be updated in the meta storage according to calculated lastScaleUpRevision,
+        // lastScaleDownRevision and causalityToken.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()
+                        && newerCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE
+                        && isScaleUpImmediate) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleDownRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleDown() != newerCfg.getDataNodesAutoAdjustScaleDown()
+                        && newerCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE
+                        && isScaleDownImmediate) {
+                    scaleDownRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleUpRevision == 0 && !olderCfg.getFilter().equals(newerCfg.getFilter())) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+            }
+
+            if ((scaleUpRevision > 0) && (scaleDownRevision > 0)) {
+                break;
+            }
+
+            entryNewerCfg = entryOlderCfg;
+        }
+
+        // The case when there is only one configuration in the history.
+        if (scaleUpRevision == 0 && entryNewerCfg != null) {
+            scaleUpRevision = entryNewerCfg.getKey();
+        }
+
+        if (scaleDownRevision == 0 && entryNewerCfg != null) {
+            scaleDownRevision = entryNewerCfg.getKey();
+        }
+
+        return new IgniteBiTuple<>(scaleUpRevision, scaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest topology event with added nodes and with removed nodes when the zone have
+     * immediate scale up and scale down timers.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownTopologyRevisions(long causalityToken, int zoneId) {
+        Set<NodeWithAttributes> newerLogicalTopology = null;
+
+        long newerTopologyRevision = 0;
+
+        Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), causalityToken);
+
+        long scaleUpTopologyRevision = 0;
+        long scaleDownTopologyRevision = 0;
+
+
+        if (!topologyEntry.empty()) {
+            byte[] newerLogicalTopologyBytes = topologyEntry.value();
+
+            newerLogicalTopology = fromBytes(newerLogicalTopologyBytes);
+
+            newerTopologyRevision = topologyEntry.revision();
+
+            while ((scaleUpTopologyRevision == 0) || (scaleDownTopologyRevision == 0)) {
+                topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), newerTopologyRevision - 1);
+
+                if (!topologyEntry.empty() && newerTopologyRevision == topologyEntry.revision()) {
+                    break;
+                }
+
+                Set<NodeWithAttributes> olderLogicalTopology;
+
+                if (topologyEntry.empty()) {
+                    olderLogicalTopology = emptySet();
+                } else {
+                    byte[] olderLogicalTopologyBytes = topologyEntry.value();
+
+                    olderLogicalTopology = fromBytes(olderLogicalTopologyBytes);
+                }
+
+                Set<NodeWithAttributes> finalNewerLogicalTopology = newerLogicalTopology;
+
+                Set<Node> removedNodes =
+                        olderLogicalTopology.stream()
+                                .filter(node -> !finalNewerLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Set<Node> addedNodes =
+                        newerLogicalTopology.stream()
+                                .filter(node -> !olderLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Map.Entry<Long, ZoneConfiguration> zoneConfigurationEntry = zonesVersionedCfg.get(zoneId)
+                        .floorEntry(newerTopologyRevision);
+
+                if (zoneConfigurationEntry == null) {
+                    break;
+                }
+
+                ZoneConfiguration zoneCfg = zoneConfigurationEntry.getValue();
+
+                if (scaleUpTopologyRevision == 0
+                        && !addedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE) {
+                    scaleUpTopologyRevision = newerTopologyRevision;
+                }
+
+                if (scaleDownTopologyRevision == 0
+                        && !removedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE) {
+                    scaleDownTopologyRevision = newerTopologyRevision;
+                }
+
+                newerLogicalTopology = olderLogicalTopology;
+
+                newerTopologyRevision = topologyEntry.revision();
+
+                if (topologyEntry.empty()) {
+                    break;
+                }
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpTopologyRevision, scaleDownTopologyRevision);
+    }
+
+    /**
+     * Search a value of zoneScaleUpChangeTriggerKey/zoneScaleDownChangeTriggerKey which equals or greater than scaleRevision.
+     * It iterates over the entries in the local metastorage. If there is an entry with a value equal to or greater than
+     * the scaleRevision, then it returns the revision of that entry. If there is no such entry then it returns zero.
+     *
+     * @param scaleRevision Scale revision.
+     * @param zoneId Zone id.
+     * @param triggerKey Trigger key.
+     * @return Future with the revision.
+     */
+    private long searchTriggerKey(Long scaleRevision, int zoneId, ByteArray triggerKey) {
+        System.out.println("waitTriggerKey " + scaleRevision + " " + zoneId);
+
+        System.out.println("waitTriggerKey after registerExactWatch");
+
+        // Must first register the watch listener to listen to new entries. Then read entries from scaleRevision to appliedRevision() + 2.
+        // In this case, we are guaranteed to get all entries from the start revision.
+        long revAfterRegister = msManager.appliedRevision() + 2;
+
+        System.out.println("waitTriggerKey revAfterRegister " + revAfterRegister);
+
+        long upperRevision = max(revAfterRegister, scaleRevision);
+
+        // Gets old entries from storage to check if the expected value was handled before watch listener was registered.
+        List<Entry> entryList = msManager.getLocally(triggerKey.bytes(), scaleRevision, upperRevision);

Review Comment:
   Why this method is deprecated? Maybe it shouldn't be...



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Asynchronously gets data nodes of the zone using causality token.
+     *
+     * <p>The returned future can be completed with {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        // Get the last scaleUp and scaleDown revisions.
+        if (isZoneRemoved) {
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        // At the dataNodesRevision the zone was created but the data nodes value had not updated yet.
+        // So the data nodes value will be equals to the logical topology on the dataNodesRevision.
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        TreeMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        if (zoneState != null) {
+            subAugmentationMap = new TreeMap<>(zoneState.topologyAugmentationMap()
+                    .subMap(0L, false, causalityToken, true));
+        }
+
+        // Wait if needed when the data nodes value will be updated in the meta storage according to calculated lastScaleUpRevision,
+        // lastScaleDownRevision and causalityToken.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()
+                        && newerCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE
+                        && isScaleUpImmediate) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleDownRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleDown() != newerCfg.getDataNodesAutoAdjustScaleDown()
+                        && newerCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE
+                        && isScaleDownImmediate) {
+                    scaleDownRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleUpRevision == 0 && !olderCfg.getFilter().equals(newerCfg.getFilter())) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+            }
+
+            if ((scaleUpRevision > 0) && (scaleDownRevision > 0)) {
+                break;
+            }
+
+            entryNewerCfg = entryOlderCfg;
+        }
+
+        // The case when there is only one configuration in the history.
+        if (scaleUpRevision == 0 && entryNewerCfg != null) {
+            scaleUpRevision = entryNewerCfg.getKey();
+        }
+
+        if (scaleDownRevision == 0 && entryNewerCfg != null) {
+            scaleDownRevision = entryNewerCfg.getKey();
+        }
+
+        return new IgniteBiTuple<>(scaleUpRevision, scaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest topology event with added nodes and with removed nodes when the zone have
+     * immediate scale up and scale down timers.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownTopologyRevisions(long causalityToken, int zoneId) {
+        Set<NodeWithAttributes> newerLogicalTopology = null;
+
+        long newerTopologyRevision = 0;
+
+        Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), causalityToken);
+
+        long scaleUpTopologyRevision = 0;
+        long scaleDownTopologyRevision = 0;
+
+
+        if (!topologyEntry.empty()) {
+            byte[] newerLogicalTopologyBytes = topologyEntry.value();
+
+            newerLogicalTopology = fromBytes(newerLogicalTopologyBytes);
+
+            newerTopologyRevision = topologyEntry.revision();
+
+            while ((scaleUpTopologyRevision == 0) || (scaleDownTopologyRevision == 0)) {
+                topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), newerTopologyRevision - 1);
+
+                if (!topologyEntry.empty() && newerTopologyRevision == topologyEntry.revision()) {
+                    break;
+                }
+
+                Set<NodeWithAttributes> olderLogicalTopology;
+
+                if (topologyEntry.empty()) {
+                    olderLogicalTopology = emptySet();
+                } else {
+                    byte[] olderLogicalTopologyBytes = topologyEntry.value();
+
+                    olderLogicalTopology = fromBytes(olderLogicalTopologyBytes);
+                }
+
+                Set<NodeWithAttributes> finalNewerLogicalTopology = newerLogicalTopology;
+
+                Set<Node> removedNodes =
+                        olderLogicalTopology.stream()
+                                .filter(node -> !finalNewerLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Set<Node> addedNodes =
+                        newerLogicalTopology.stream()
+                                .filter(node -> !olderLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Map.Entry<Long, ZoneConfiguration> zoneConfigurationEntry = zonesVersionedCfg.get(zoneId)
+                        .floorEntry(newerTopologyRevision);
+
+                if (zoneConfigurationEntry == null) {
+                    break;
+                }
+
+                ZoneConfiguration zoneCfg = zoneConfigurationEntry.getValue();
+
+                if (scaleUpTopologyRevision == 0
+                        && !addedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE) {
+                    scaleUpTopologyRevision = newerTopologyRevision;
+                }
+
+                if (scaleDownTopologyRevision == 0
+                        && !removedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE) {
+                    scaleDownTopologyRevision = newerTopologyRevision;
+                }
+
+                newerLogicalTopology = olderLogicalTopology;
+
+                newerTopologyRevision = topologyEntry.revision();
+
+                if (topologyEntry.empty()) {
+                    break;
+                }
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpTopologyRevision, scaleDownTopologyRevision);
+    }
+
+    /**
+     * Search a value of zoneScaleUpChangeTriggerKey/zoneScaleDownChangeTriggerKey which equals or greater than scaleRevision.
+     * It iterates over the entries in the local metastorage. If there is an entry with a value equal to or greater than
+     * the scaleRevision, then it returns the revision of that entry. If there is no such entry then it returns zero.
+     *
+     * @param scaleRevision Scale revision.
+     * @param zoneId Zone id.
+     * @param triggerKey Trigger key.
+     * @return Future with the revision.
+     */
+    private long searchTriggerKey(Long scaleRevision, int zoneId, ByteArray triggerKey) {
+        System.out.println("waitTriggerKey " + scaleRevision + " " + zoneId);
+
+        System.out.println("waitTriggerKey after registerExactWatch");
+
+        // Must first register the watch listener to listen to new entries. Then read entries from scaleRevision to appliedRevision() + 2.
+        // In this case, we are guaranteed to get all entries from the start revision.
+        long revAfterRegister = msManager.appliedRevision() + 2;
+
+        System.out.println("waitTriggerKey revAfterRegister " + revAfterRegister);
+
+        long upperRevision = max(revAfterRegister, scaleRevision);
+
+        // Gets old entries from storage to check if the expected value was handled before watch listener was registered.
+        List<Entry> entryList = msManager.getLocally(triggerKey.bytes(), scaleRevision, upperRevision);
+
+        long revision = 0;
+
+        for (Entry entry : entryList) {
+            System.out.println("waitTriggerKey iteration revision " + entry.value());
+
+            // scaleRevision is null if the zone was removed.
+            if (scaleRevision == null) {
+                if (entry.value() == null) {
+                    revision = entry.revision();
+
+                    break;
+                }
+            } else {
+                long entryValue = bytesToLong(entry.value());
+
+                if (entryValue >= scaleRevision) {
+                    revision = entry.revision();
+
+                    break;
+                }
+            }
+        }
+
+        return revision;
+    }
+
+    /**
+     * causalityOnUpdateScaleUp.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleUp newScaleUp.
+     */
+    public void causalityOnUpdateScaleUp(long revision, int zoneId, int newScaleUp) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleUp(newScaleUp);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * causalityOnUpdateScaleDown.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleDown newScaleDown.
+     */
+    public void causalityOnUpdateScaleDown(long revision, int zoneId, int newScaleDown) {

Review Comment:
   This method and similar ones mostly repeat each other, except one line. They can be merged to 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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneCausalityDataNodesTest.java:
##########
@@ -0,0 +1,683 @@
+/*
+ * 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 java.util.Collections.emptySet;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.cluster.management.topology.LogicalTopologyImpl.LOGICAL_TOPOLOGY_KEY;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneLogicalTopologyPrefix;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesDataNodesPrefix;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyVersionKey;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.IgniteUtils.startsWith;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologySnapshot;
+import org.apache.ignite.internal.distributionzones.DistributionZoneConfigurationParameters.Builder;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.exception.DistributionZoneNotFoundException;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.EntryEvent;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for causality data nodes updating in {@link DistributionZoneManager}.
+ */
+public class DistributionZoneCausalityDataNodesTest extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME_1 = "zone1";
+
+    private static final String ZONE_NAME_2 = "zone2";
+
+    private static final int ZONE_ID_1 = 1;
+
+    private static final int ZONE_ID_2 = 2;
+
+    private static final LogicalNode NODE_0 =
+            new LogicalNode("node_id_0", "node_name_0", new NetworkAddress("localhost", 123));
+
+    private static final LogicalNode NODE_1 =
+            new LogicalNode("node_id_1", "node_name_1", new NetworkAddress("localhost", 123));
+
+    private static final LogicalNode NODE_2 =
+            new LogicalNode("node_id_2", "node_name_2", new NetworkAddress("localhost", 123));
+
+    /**
+     * Contains futures that is completed when the topology watch listener receive the event with expected data nodes.

Review Comment:
   Fixed



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneCausalityDataNodesTest.java:
##########
@@ -0,0 +1,683 @@
+/*
+ * 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 java.util.Collections.emptySet;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.cluster.management.topology.LogicalTopologyImpl.LOGICAL_TOPOLOGY_KEY;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneLogicalTopologyPrefix;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesDataNodesPrefix;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyVersionKey;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.IgniteUtils.startsWith;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
+import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologySnapshot;
+import org.apache.ignite.internal.distributionzones.DistributionZoneConfigurationParameters.Builder;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.exception.DistributionZoneNotFoundException;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.EntryEvent;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for causality data nodes updating in {@link DistributionZoneManager}.
+ */
+public class DistributionZoneCausalityDataNodesTest extends BaseDistributionZoneManagerTest {
+    private static final String ZONE_NAME_1 = "zone1";
+
+    private static final String ZONE_NAME_2 = "zone2";
+
+    private static final int ZONE_ID_1 = 1;
+
+    private static final int ZONE_ID_2 = 2;
+
+    private static final LogicalNode NODE_0 =
+            new LogicalNode("node_id_0", "node_name_0", new NetworkAddress("localhost", 123));
+
+    private static final LogicalNode NODE_1 =
+            new LogicalNode("node_id_1", "node_name_1", new NetworkAddress("localhost", 123));
+
+    private static final LogicalNode NODE_2 =
+            new LogicalNode("node_id_2", "node_name_2", new NetworkAddress("localhost", 123));
+
+    /**
+     * Contains futures that is completed when the topology watch listener receive the event with expected data nodes.
+     * Mapping of node names -> future with event revision.
+     */
+    private final ConcurrentHashMap<Set<String>, CompletableFuture<Long>> topologyRevisions = new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the data nodes watch listener receive the event with expected zone id and data nodes.
+     * Mapping of zone id and node names -> future with event revision.
+     */
+    private final ConcurrentHashMap<IgniteBiTuple<Integer, Set<String>>, CompletableFuture<Long>> zoneDataNodesRevisions =
+            new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the scale up update listener receive the event with expected zone id.
+     * Mapping of zone id -> future with event revision.
+     */
+    private final ConcurrentHashMap<Integer, CompletableFuture<Long>> zoneScaleUpRevisions = new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the scale down update listener receive the event with expected zone id.
+     * Mapping of zone id -> future with event revision.
+     */
+    private final ConcurrentHashMap<Integer, CompletableFuture<Long>> zoneScaleDownRevisions = new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the zone configuration listener receive the zone creation event with expected zone id.
+     * Mapping of zone id -> future with event revision.
+     */
+    private final ConcurrentHashMap<Integer, CompletableFuture<Long>> createZoneRevisions = new ConcurrentHashMap<>();
+
+    /**
+     * Contains futures that is completed when the zone configuration listener receive the zone dropping event with expected zone id.
+     * Mapping of zone id -> future with event revision.
+     */
+    private final ConcurrentHashMap<Integer, CompletableFuture<Long>> dropZoneRevisions = new ConcurrentHashMap<>();
+
+    @BeforeEach
+    void beforeEach() throws NodeStoppingException {
+        metaStorageManager.registerPrefixWatch(zoneLogicalTopologyPrefix(), createMetastorageTopologyListener());
+
+        metaStorageManager.registerPrefixWatch(zonesDataNodesPrefix(), createMetastorageDataNodesListener());
+
+        ZonesConfigurationListener zonesConfigurationListener = new ZonesConfigurationListener();
+
+        zonesConfiguration.distributionZones().listenElements(zonesConfigurationListener);
+        zonesConfiguration.distributionZones().any().dataNodesAutoAdjustScaleUp().listen(onUpdateScaleUp());
+        zonesConfiguration.distributionZones().any().dataNodesAutoAdjustScaleDown().listen(onUpdateScaleDown());
+
+        zonesConfiguration.defaultDistributionZone().listen(zonesConfigurationListener);
+        zonesConfiguration.defaultDistributionZone().dataNodesAutoAdjustScaleUp().listen(onUpdateScaleUp());
+        zonesConfiguration.defaultDistributionZone().dataNodesAutoAdjustScaleDown().listen(onUpdateScaleDown());
+
+        distributionZoneManager.start();
+
+        metaStorageManager.deployWatches();
+    }
+
+    /**
+     * Tests data nodes updating on a topology leap.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void topologyLeapUpdate() throws Exception {
+        // Prerequisite.
+
+        // Create the zone with immediate timers.
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_1)
+                                .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                                .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Create the zone with not immediate timers.
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_2)
+                                .dataNodesAutoAdjustScaleUp(1)
+                                .dataNodesAutoAdjustScaleDown(1)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Create logical topology with NODE_0 and NODE_1.
+        topology.putNode(NODE_0);
+
+        Set<LogicalNode> twoNodes1 = Set.of(NODE_0, NODE_1);
+        Set<String> twoNodesNames1 = Set.of(NODE_0.name(), NODE_1.name());
+
+        CompletableFuture<Long> dataNodesUpdateRevision = getZoneDataNodesRevision(ZONE_ID_2, twoNodes1);
+
+        // Check that data nodes value of both zone is NODE_0 and NODE_1.
+        long topologyRevision1 = putNodeInLogicalTopologyAndGetRevision(NODE_1, twoNodes1);
+
+        CompletableFuture<Set<String>> dataNodesFut0 = distributionZoneManager.dataNodes(topologyRevision1, ZONE_ID_1);
+        assertThat(dataNodesFut0, willBe(twoNodesNames1));
+
+        long dataNodesRevisionZone = dataNodesUpdateRevision.get(3, SECONDS);
+
+        CompletableFuture<Set<String>> dataNodesFut1 = distributionZoneManager.dataNodes(dataNodesRevisionZone, ZONE_ID_2);
+        assertThat(dataNodesFut1, willBe(twoNodesNames1));
+
+        // Test steps.
+
+        // Change logical topology. NODE_1 is left. NODE_2 is added.
+        Set<LogicalNode> twoNodes2 = Set.of(NODE_0, NODE_2);
+        Set<String> twoNodesNames2 = Set.of(NODE_0.name(), NODE_2.name());
+
+        dataNodesUpdateRevision = getZoneDataNodesRevision(ZONE_ID_2, twoNodes2);
+
+        long topologyRevision2 = fireTopologyLeapAndGetRevision(twoNodes2);
+
+        // Check that data nodes value of the zone with immediate timers with the topology update revision is NODE_0 and NODE_2.
+        CompletableFuture<Set<String>> dataNodesFut3 = distributionZoneManager.dataNodes(topologyRevision2, ZONE_ID_1);
+        assertThat(dataNodesFut3, willBe(twoNodesNames2));
+
+        // Check that data nodes value of the zone with not immediate timers with the topology update revision is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut4 = distributionZoneManager.dataNodes(topologyRevision2, ZONE_ID_2);
+        assertThat(dataNodesFut4, willBe(twoNodesNames1));
+
+        // Check that data nodes value of the zone with not immediate timers with the data nodes update revision is NODE_0 and NODE_2.
+        dataNodesRevisionZone = dataNodesUpdateRevision.get(3, SECONDS);
+        CompletableFuture<Set<String>> dataNodesFut5 = distributionZoneManager.dataNodes(dataNodesRevisionZone, ZONE_ID_2);
+        assertThat(dataNodesFut5, willBe(twoNodesNames2));
+    }
+
+    /**
+     * Tests data nodes updating on a scale up changing.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void dataNodesUpdatedAfterScaleUpChanged() throws Exception {
+        // Prerequisite.
+
+        // Create the zone with immediate scale up timer and immediate scale down timer.
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_1)
+                                .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                                .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Create logical topology with NODE_0.
+        Set<LogicalNode> oneNode = Set.of(NODE_0);
+        Set<String> oneNodeName = Set.of(NODE_0.name());
+
+        long topologyRevision1 = putNodeInLogicalTopologyAndGetRevision(NODE_1, oneNode);
+
+        // Check that data nodes value of the the zone is NODE_0.
+        CompletableFuture<Set<String>> dataNodesFut1 = distributionZoneManager.dataNodes(topologyRevision1, ZONE_ID_1);
+        assertThat(dataNodesFut1, willBe(oneNodeName));
+
+        // Changes a scale up timer to not immediate.
+        distributionZoneManager.alterZone(
+                        ZONE_NAME_1,
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_1)
+                                .dataNodesAutoAdjustScaleUp(10000)
+                                .dataNodesAutoAdjustScaleDown(IMMEDIATE_TIMER_VALUE)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Test steps.
+
+        // Change logical topology. NODE_1 is added.
+        Set<LogicalNode> twoNodes = Set.of(NODE_0, NODE_1);
+        Set<String> twoNodesNames = Set.of(NODE_0.name(), NODE_1.name());
+
+        long topologyRevision2 = putNodeInLogicalTopologyAndGetRevision(NODE_1, twoNodes);
+
+        // Check that data nodes value of the zone with the topology update revision is NODE_0 because scale up timer has not fired yet.
+        CompletableFuture<Set<String>> dataNodesFut2 = distributionZoneManager.dataNodes(topologyRevision2, ZONE_ID_1);
+        assertThat(dataNodesFut2, willBe(oneNode));
+
+        // Change scale up value to immediate.
+        long scaleUpRevision = alterZoneScaleUpAndGetRevision(ZONE_NAME_1, IMMEDIATE_TIMER_VALUE);
+
+        // Check that data nodes value of the zone with the scale up update revision is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut3 = distributionZoneManager.dataNodes(scaleUpRevision, ZONE_ID_1);
+        assertThat(dataNodesFut3, willBe(twoNodesNames));
+    }
+
+    /**
+     * Tests data nodes updating on a scale down changing.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void dataNodesUpdatedAfterScaleDownChanged() throws Exception {
+        // Prerequisite.
+
+        // Create the zone with immediate scale up timer and not immediate scale down timer.
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME_1)
+                                .dataNodesAutoAdjustScaleUp(IMMEDIATE_TIMER_VALUE)
+                                .dataNodesAutoAdjustScaleDown(10000)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        // Create logical topology with NODE_0 and NODE_1.
+        topology.putNode(NODE_0);
+
+        Set<LogicalNode> twoNodes = Set.of(NODE_0, NODE_1);
+        Set<String> twoNodesNames = Set.of(NODE_0.name(), NODE_1.name());
+
+        long topologyRevision1 = putNodeInLogicalTopologyAndGetRevision(NODE_1, twoNodes);
+
+        // Check that data nodes value of the the zone is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut1 = distributionZoneManager.dataNodes(topologyRevision1, ZONE_ID_1);
+        assertThat(dataNodesFut1, willBe(twoNodesNames));
+
+        // Test steps.
+
+        // Change logical topology. NODE_1 is added.
+        Set<LogicalNode> oneNode = Set.of(NODE_0);
+        Set<String> oneNodeName = Set.of(NODE_0.name());
+
+        long topologyRevision2 = removeNodeInLogicalTopologyAndGetRevision(Set.of(NODE_1), oneNode);
+
+        // Check that data nodes value of the zone with the topology update revision is NODE_0 because scale down timer has not fired yet.
+        CompletableFuture<Set<String>> dataNodesFut2 = distributionZoneManager.dataNodes(topologyRevision2, ZONE_ID_1);
+        assertThat(dataNodesFut2, willBe(twoNodesNames));
+
+        // Change scale down value to immediate.
+        long scaleDownRevision = alterZoneScaleDownAndGetRevision(ZONE_NAME_1, IMMEDIATE_TIMER_VALUE);
+
+        // Check that data nodes value of the zone with the scale down update revision is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut3 = distributionZoneManager.dataNodes(scaleDownRevision, ZONE_ID_1);
+        assertThat(dataNodesFut3, willBe(oneNodeName));
+    }
+
+    /**
+     * Tests data nodes obtaining with revision before a zone creation and after a zone dropping.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    void createThenDropZone() throws Exception {
+        // Prerequisite.
+
+        // Create logical topology with NODE_0 and NODE_1.
+        topology.putNode(NODE_0);
+        topology.putNode(NODE_1);
+
+        Set<String> twoNodesNames = Set.of(NODE_0.name(), NODE_1.name());
+
+        // Test steps.
+
+        // Create a zone.
+        long createZoneRevision = createZoneAndGetRevision(ZONE_NAME_1, ZONE_ID_1, IMMEDIATE_TIMER_VALUE, IMMEDIATE_TIMER_VALUE);
+
+        // Check that data nodes value of the zone with the revision lower than the create zone revision is absent.
+        CompletableFuture<Set<String>> dataNodesFut1 = distributionZoneManager.dataNodes(createZoneRevision - 1, ZONE_ID_1);
+        assertThrows(DistributionZoneNotFoundException.class, () -> dataNodesFut1.get(3, SECONDS));
+
+        // Check that data nodes value of the zone with the create zone revision is NODE_0 and NODE_1.
+        CompletableFuture<Set<String>> dataNodesFut2 = distributionZoneManager.dataNodes(createZoneRevision, ZONE_ID_1);
+        assertThat(dataNodesFut2, willBe(twoNodesNames));
+
+        // Drop the zone.
+        long dropZoneRevision = dropZoneAndGetRevision(ZONE_NAME_1);
+
+        // Check that data nodes value of the zone with the drop zone revision is absent.
+        CompletableFuture<Set<String>> dataNodesFut3 = distributionZoneManager.dataNodes(dropZoneRevision, ZONE_ID_1);
+        assertThrows(DistributionZoneNotFoundException.class, () -> dataNodesFut3.get(3, SECONDS));
+    }
+
+    /**
+     * Puts a given node as a part of the logical topology and return revision of a topology watch listener event.
+     *
+     * @param node Node to put.
+     * @param expectedTopology Expected topology for future completing.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long putNodeInLogicalTopologyAndGetRevision(
+            LogicalNode node,
+            Set<LogicalNode> expectedTopology
+    ) throws Exception {
+        Set<String> nodeNames = expectedTopology.stream().map(ClusterNode::name).collect(toSet());
+
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        topologyRevisions.put(nodeNames, revisionFut);
+
+        topology.putNode(node);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Removes given nodes from the logical topology and return revision of a topology watch listener event.
+     *
+     * @param nodes Nodes to remove.
+     * @param expectedTopology Expected topology for future completing.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long removeNodeInLogicalTopologyAndGetRevision(
+            Set<LogicalNode> nodes,
+            Set<LogicalNode> expectedTopology
+    ) throws Exception {
+        Set<String> nodeNames = expectedTopology.stream().map(ClusterNode::name).collect(toSet());
+
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        topologyRevisions.put(nodeNames, revisionFut);
+
+        topology.removeNodes(nodes);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Changes data nodes in logical topology and return revision of a topology watch listener event.
+     *
+     * @param nodes Nodes to remove.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long fireTopologyLeapAndGetRevision(Set<LogicalNode> nodes) throws Exception {
+        Set<String> nodeNames = nodes.stream().map(ClusterNode::name).collect(toSet());
+
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        topologyRevisions.put(nodeNames, revisionFut);
+
+        long topVer = topology.getLogicalTopology().version() + 1;
+
+        clusterStateStorage.put(LOGICAL_TOPOLOGY_KEY, ByteUtils.toBytes(new LogicalTopologySnapshot(topVer, nodes)));
+
+        topology.fireTopologyLeap();
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Changes a scale up timer value of a zone and return the revision of a zone update event.
+     *
+     * @param zoneName Zone name.
+     * @param scaleUp New scale up value.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long alterZoneScaleUpAndGetRevision(String zoneName, int scaleUp) throws Exception {
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        int zoneId = distributionZoneManager.getZoneId(zoneName);
+
+        zoneScaleUpRevisions.put(zoneId, revisionFut);
+
+        distributionZoneManager.alterZone(zoneName, new Builder(zoneName)
+                        .dataNodesAutoAdjustScaleUp(scaleUp).build())
+                .get(3, SECONDS);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Changes a scale down timer value of a zone and return the revision of a zone update event.
+     *
+     * @param zoneName Zone name.
+     * @param scaleDown New scale down value.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long alterZoneScaleDownAndGetRevision(String zoneName, int scaleDown) throws Exception {
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        int zoneId = distributionZoneManager.getZoneId(zoneName);
+
+        zoneScaleDownRevisions.put(zoneId, revisionFut);
+
+        distributionZoneManager.alterZone(zoneName, new Builder(zoneName)
+                        .dataNodesAutoAdjustScaleDown(scaleDown).build())
+                .get(3, SECONDS);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Creates a zone and return the revision of a create zone event.
+     *
+     * @param zoneName Zone name.
+     * @param zoneId Zone id.
+     * @param scaleUp Scale up value.
+     * @param scaleDown Scale down value.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long createZoneAndGetRevision(String zoneName, int zoneId, int scaleUp, int scaleDown) throws Exception {
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        createZoneRevisions.put(zoneId, revisionFut);
+
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(zoneName)
+                                .dataNodesAutoAdjustScaleUp(scaleUp)
+                                .dataNodesAutoAdjustScaleDown(scaleDown)
+                                .build()
+                )
+                .get(3, SECONDS);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Drops a zone and return the revision of a drop zone event.
+     *
+     * @param zoneName Zone name.
+     * @return Revision.
+     * @throws Exception If failed.
+     */
+    private long dropZoneAndGetRevision(String zoneName) throws Exception {
+        CompletableFuture<Long> revisionFut = new CompletableFuture<>();
+
+        int zoneId = distributionZoneManager.getZoneId(zoneName);
+
+        dropZoneRevisions.put(zoneId, revisionFut);
+
+        distributionZoneManager.dropZone(zoneName).get(3, SECONDS);
+
+        return revisionFut.get(3, SECONDS);
+    }
+
+    /**
+     * Removes given nodes from the logical topology and return future with revision of topology watch listener event.
+     *
+     * @param zoneId Zone id.
+     * @param nodes Expected data nodes.
+     * @return Future with revision.
+     */
+    private CompletableFuture<Long> getZoneDataNodesRevision(int zoneId, Set<LogicalNode> nodes) {

Review Comment:
   Fixed



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -1793,6 +1859,7 @@ public static class ZoneState {
             this.executor = executor;
             topologyAugmentationMap = new ConcurrentSkipListMap<>();
             nodes = emptySet();
+            this.zoneId = zoneId;

Review Comment:
   I removed it.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;

Review Comment:
   This map is not cleanable now. It will be fixed later in other ticket, I will create it. We discussed it earlier.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -705,219 +705,230 @@ private CompletableFuture<?> onTableDelete(ConfigurationNotificationEvent<TableV
      */
     private CompletableFuture<?> createTablePartitionsLocally(
             long causalityToken,
-            List<Set<Assignment>> assignments,
+            CompletableFuture<List<Set<Assignment>>> assignments,
             int zoneId,
             TableImpl table
     ) {
         int tableId = table.tableId();
 
-        List<Set<Assignment>> newAssignments = assignments;
+        // Create new raft nodes according to new assignments.
+        Supplier<CompletableFuture<Void>> updateAssignmentsClosure = () -> {
+            return assignments.thenCompose(newAssignments -> {
+                // Empty assignments might be a valid case if tables are created from within cluster init HOCON
+                // configuration, which is not supported now.
+                assert newAssignments != null : IgniteStringFormatter.format("Table [id={}] has empty assignments.", tableId);
+
+                int partitions = newAssignments.size();
 
-        // Empty assignments might be a valid case if tables are created from within cluster init HOCON
-        // configuration, which is not supported now.
-        assert newAssignments != null : IgniteStringFormatter.format("Table [id={}] has empty assignments.", tableId);
+                CompletableFuture<?>[] futures = new CompletableFuture<?>[partitions];
 
-        int partitions = newAssignments.size();
+                // TODO: https://issues.apache.org/jira/browse/IGNITE-19713 Process assignments and set partitions only for assigned partitions.
+                PartitionSet parts = new BitSetPartitionSet();
 
-        CompletableFuture<?>[] futures = new CompletableFuture<?>[partitions];
+                for (int i = 0; i < futures.length; i++) {
+                    futures[i] = new CompletableFuture<>();
 
-        // TODO: https://issues.apache.org/jira/browse/IGNITE-19713 Process assignments and set partitions only for assigned partitions.
-        PartitionSet parts = new BitSetPartitionSet();
+                    parts.set(i);
+                }
 
-        for (int i = 0; i < futures.length; i++) {
-            futures[i] = new CompletableFuture<>();
+                String localMemberName = localNode().name();
 
-            parts.set(i);
-        }
+                for (int i = 0; i < partitions; i++) {
+                    int partId = i;
 
-        String localMemberName = localNode().name();
+                    Set<Assignment> newPartAssignment = newAssignments.get(partId);
 
-        // Create new raft nodes according to new assignments.
-        Supplier<CompletableFuture<Void>> updateAssignmentsClosure = () -> {
-            for (int i = 0; i < partitions; i++) {
-                int partId = i;
+                    InternalTable internalTbl = table.internalTable();
 
-                Set<Assignment> newPartAssignment = newAssignments.get(partId);
+                    Assignment localMemberAssignment = newPartAssignment.stream()
+                            .filter(a -> a.consistentId().equals(localMemberName))
+                            .findAny()
+                            .orElse(null);
 
-                InternalTable internalTbl = table.internalTable();
+                    PeersAndLearners newConfiguration = configurationFromAssignments(newPartAssignment);
 
-                Assignment localMemberAssignment = newPartAssignment.stream()
-                        .filter(a -> a.consistentId().equals(localMemberName))
-                        .findAny()
-                        .orElse(null);
+                    TablePartitionId replicaGrpId = new TablePartitionId(tableId, partId);
 
-                PeersAndLearners newConfiguration = configurationFromAssignments(newPartAssignment);
+                    placementDriver.updateAssignment(replicaGrpId, newConfiguration.peers().stream().map(Peer::consistentId)
+                            .collect(toList()));
 
-                TablePartitionId replicaGrpId = new TablePartitionId(tableId, partId);
+                    var safeTimeTracker = new PendingComparableValuesTracker<HybridTimestamp, Void>(
+                            new HybridTimestamp(1, 0)
+                    );
+                    var storageIndexTracker = new PendingComparableValuesTracker<Long, Void>(0L);
 
-                placementDriver.updateAssignment(replicaGrpId, newConfiguration.peers().stream().map(Peer::consistentId)
-                        .collect(toList()));
+                    ((InternalTableImpl) internalTbl).updatePartitionTrackers(partId, safeTimeTracker, storageIndexTracker);
 
-                var safeTimeTracker = new PendingComparableValuesTracker<HybridTimestamp, Void>(
-                        new HybridTimestamp(1, 0)
-                );
-                var storageIndexTracker = new PendingComparableValuesTracker<Long, Void>(0L);
+                    PartitionStorages partitionStorages = getPartitionStorages(table, partId);
 
-                ((InternalTableImpl) internalTbl).updatePartitionTrackers(partId, safeTimeTracker, storageIndexTracker);
+                    PartitionDataStorage partitionDataStorage = partitionDataStorage(partitionStorages.getMvPartitionStorage(),
+                            internalTbl, partId);
 
-                PartitionStorages partitionStorages = getPartitionStorages(table, partId);
+                    PartitionUpdateHandlers partitionUpdateHandlers = createPartitionUpdateHandlers(
+                            partId,
+                            partitionDataStorage,
+                            table,
+                            safeTimeTracker
+                    );
 
-                PartitionDataStorage partitionDataStorage = partitionDataStorage(partitionStorages.getMvPartitionStorage(),
-                        internalTbl, partId);
+                    mvGc.addStorage(replicaGrpId, partitionUpdateHandlers.gcUpdateHandler);
 
-                PartitionUpdateHandlers partitionUpdateHandlers = createPartitionUpdateHandlers(
-                        partId,
-                        partitionDataStorage,
-                        table,
-                        safeTimeTracker
-                );
+                    CompletableFuture<Boolean> startGroupFut;
 
-                mvGc.addStorage(replicaGrpId, partitionUpdateHandlers.gcUpdateHandler);
+                    // start new nodes, only if it is table creation, other cases will be covered by rebalance logic
+                    if (localMemberAssignment != null) {
+                        CompletableFuture<Boolean> shouldStartGroupFut;
 
-                CompletableFuture<Boolean> startGroupFut;
+                        // If Raft is running in in-memory mode or the PDS has been cleared, we need to remove the current node
+                        // from the Raft group in order to avoid the double vote problem.
+                        // <MUTED> See https://issues.apache.org/jira/browse/IGNITE-16668 for details.
+                        // TODO: https://issues.apache.org/jira/browse/IGNITE-19046 Restore "|| !hasData"
+                        if (internalTbl.storage().isVolatile()) {
+                            shouldStartGroupFut = queryDataNodesCount(tableId, partId, newConfiguration.peers())
+                                    .thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
 
-                // start new nodes, only if it is table creation, other cases will be covered by rebalance logic
-                if (localMemberAssignment != null) {
-                    CompletableFuture<Boolean> shouldStartGroupFut;
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
 
-                    // If Raft is running in in-memory mode or the PDS has been cleared, we need to remove the current node
-                    // from the Raft group in order to avoid the double vote problem.
-                    // <MUTED> See https://issues.apache.org/jira/browse/IGNITE-16668 for details.
-                    // TODO: https://issues.apache.org/jira/browse/IGNITE-19046 Restore "|| !hasData"
-                    if (internalTbl.storage().isVolatile()) {
-                        shouldStartGroupFut = queryDataNodesCount(tableId, partId, newConfiguration.peers()).thenApply(dataNodesCount -> {
-                            boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (newConfiguration.peers().size() / 2) + 1;
 
-                            if (fullPartitionRestart) {
-                                return true;
-                            }
+                                        if (majorityAvailable) {
+                                            RebalanceUtil.startPeerRemoval(replicaGrpId, localMemberAssignment, metaStorageMgr);
 
-                            boolean majorityAvailable = dataNodesCount >= (newConfiguration.peers().size() / 2) + 1;
+                                            return false;
+                                        } else {
+                                            // No majority and not a full partition restart - need to restart nodes
+                                            // with current partition.
+                                            String msg = "Unable to start partition " + partId + ". Majority not available.";
 
-                            if (majorityAvailable) {
-                                RebalanceUtil.startPeerRemoval(replicaGrpId, localMemberAssignment, metaStorageMgr);
+                                            throw new IgniteInternalException(msg);
+                                        }
+                                    });
+                        } else {
+                            shouldStartGroupFut = completedFuture(true);
+                        }
 
+                        startGroupFut = shouldStartGroupFut.thenApplyAsync(startGroup -> inBusyLock(busyLock, () -> {
+                            if (!startGroup) {
                                 return false;
-                            } else {
-                                // No majority and not a full partition restart - need to restart nodes
-                                // with current partition.
-                                String msg = "Unable to start partition " + partId + ". Majority not available.";
+                            }
+                            TxStateStorage txStatePartitionStorage = partitionStorages.getTxStateStorage();
+
+                            RaftGroupOptions groupOptions = groupOptionsForPartition(
+                                    internalTbl.storage(),
+                                    internalTbl.txStateStorage(),
+                                    partitionKey(internalTbl, partId),
+                                    partitionUpdateHandlers
+                            );
+
+                            Peer serverPeer = newConfiguration.peer(localMemberName);
+
+                            var raftNodeId = new RaftNodeId(replicaGrpId, serverPeer);
+
+                            try {
+                                // TODO: use RaftManager interface, see https://issues.apache.org/jira/browse/IGNITE-18273
+                                ((Loza) raftMgr).startRaftGroupNode(
+                                        raftNodeId,
+                                        newConfiguration,
+                                        new PartitionListener(
+                                                partitionDataStorage,
+                                                partitionUpdateHandlers.storageUpdateHandler,
+                                                txStatePartitionStorage,
+                                                safeTimeTracker,
+                                                storageIndexTracker
+                                        ),
+                                        new RebalanceRaftGroupEventsListener(
+                                                metaStorageMgr,
+                                                replicaGrpId,
+                                                busyLock,
+                                                createPartitionMover(internalTbl, partId),
+                                                this::calculateAssignments,
+                                                rebalanceScheduler
+                                        ),
+                                        groupOptions
+                                );
 
-                                throw new IgniteInternalException(msg);
+                                return true;
+                            } catch (NodeStoppingException ex) {
+                                throw new CompletionException(ex);
                             }
-                        });
+                        }), ioExecutor);
                     } else {
-                        shouldStartGroupFut = completedFuture(true);
+                        startGroupFut = completedFuture(false);
                     }
 
-                    startGroupFut = shouldStartGroupFut.thenApplyAsync(startGroup -> inBusyLock(busyLock, () -> {
-                        if (!startGroup) {
-                            return false;
-                        }
-                        TxStateStorage txStatePartitionStorage = partitionStorages.getTxStateStorage();
-
-                        RaftGroupOptions groupOptions = groupOptionsForPartition(
-                                internalTbl.storage(),
-                                internalTbl.txStateStorage(),
-                                partitionKey(internalTbl, partId),
-                                partitionUpdateHandlers
-                        );
+                    startGroupFut
+                            .thenComposeAsync(v -> inBusyLock(busyLock, () -> {
+                                try {
+                                    //TODO IGNITE-19614 This procedure takes 10 seconds if there's no majority online.
+                                    return raftMgr.startRaftGroupService(replicaGrpId, newConfiguration, raftGroupServiceFactory);
+                                } catch (NodeStoppingException ex) {
+                                    return failedFuture(ex);
+                                }
+                            }), ioExecutor)
+                            .thenAcceptAsync(updatedRaftGroupService -> inBusyLock(busyLock, () -> {
+                                ((InternalTableImpl) internalTbl).updateInternalTableRaftGroupService(partId, updatedRaftGroupService);
 
-                        Peer serverPeer = newConfiguration.peer(localMemberName);
+                                boolean startedRaftNode = startGroupFut.join();
+                                if (localMemberAssignment == null || !startedRaftNode) {
+                                    return;
+                                }
 
-                        var raftNodeId = new RaftNodeId(replicaGrpId, serverPeer);
+                                MvPartitionStorage partitionStorage = partitionStorages.getMvPartitionStorage();
+                                TxStateStorage txStateStorage = partitionStorages.getTxStateStorage();
 
-                        try {
-                            // TODO: use RaftManager interface, see https://issues.apache.org/jira/browse/IGNITE-18273
-                            ((Loza) raftMgr).startRaftGroupNode(
-                                    raftNodeId,
-                                    newConfiguration,
-                                    new PartitionListener(
-                                            partitionDataStorage,
-                                            partitionUpdateHandlers.storageUpdateHandler,
-                                            txStatePartitionStorage,
-                                            safeTimeTracker,
-                                            storageIndexTracker
-                                    ),
-                                    new RebalanceRaftGroupEventsListener(
-                                            metaStorageMgr,
+                                try {
+                                    startReplicaWithNewListener(
                                             replicaGrpId,
-                                            busyLock,
-                                            createPartitionMover(internalTbl, partId),
-                                            this::calculateAssignments,
-                                            rebalanceScheduler
-                                    ),
-                                    groupOptions
-                            );
+                                            table,
+                                            safeTimeTracker,
+                                            storageIndexTracker,
+                                            partitionStorage,
+                                            txStateStorage,
+                                            partitionUpdateHandlers,
+                                            updatedRaftGroupService,
+                                            schemaManager.schemaRegistry(causalityToken, tableId)
+                                    );
+                                } catch (NodeStoppingException ex) {
+                                    throw new AssertionError("Loza was stopped before Table manager", ex);
+                                }
+                            }), ioExecutor)
+                            .whenComplete((res, ex) -> {
+                                if (ex != null) {
+                                    LOG.warn("Unable to update raft groups on the node [tableId={}, partitionId={}]", ex, tableId, partId);
 
-                            return true;
-                        } catch (NodeStoppingException ex) {
-                            throw new CompletionException(ex);
-                        }
-                    }), ioExecutor);
-                } else {
-                    startGroupFut = completedFuture(false);
+                                    futures[partId].completeExceptionally(ex);
+                                } else {
+                                    futures[partId].complete(null);
+                                }
+                            });
                 }
 
-                startGroupFut
-                        .thenComposeAsync(v -> inBusyLock(busyLock, () -> {
-                            try {
-                                //TODO IGNITE-19614 This procedure takes 10 seconds if there's no majority online.
-                                return raftMgr.startRaftGroupService(replicaGrpId, newConfiguration, raftGroupServiceFactory);
-                            } catch (NodeStoppingException ex) {
-                                return failedFuture(ex);
-                            }
-                        }), ioExecutor)
-                        .thenAcceptAsync(updatedRaftGroupService -> inBusyLock(busyLock, () -> {
-                            ((InternalTableImpl) internalTbl).updateInternalTableRaftGroupService(partId, updatedRaftGroupService);
-
-                            boolean startedRaftNode = startGroupFut.join();
-                            if (localMemberAssignment == null || !startedRaftNode) {
-                                return;
-                            }
+                return allOf(futures);
+            });
 
-                            MvPartitionStorage partitionStorage = partitionStorages.getMvPartitionStorage();
-                            TxStateStorage txStateStorage = partitionStorages.getTxStateStorage();
 
-                            try {
-                                startReplicaWithNewListener(
-                                        replicaGrpId,
-                                        table,
-                                        safeTimeTracker,
-                                        storageIndexTracker,
-                                        partitionStorage,
-                                        txStateStorage,
-                                        partitionUpdateHandlers,
-                                        updatedRaftGroupService,
-                                        schemaManager.schemaRegistry(causalityToken, tableId)
-                                );
-                            } catch (NodeStoppingException ex) {
-                                throw new AssertionError("Loza was stopped before Table manager", ex);
-                            }
-                        }), ioExecutor)
-                        .whenComplete((res, ex) -> {
-                            if (ex != null) {
-                                LOG.warn("Unable to update raft groups on the node [tableId={}, partitionId={}]", ex, tableId, partId);
-
-                                futures[partId].completeExceptionally(ex);
-                            } else {
-                                futures[partId].complete(null);
-                            }
-                        });
-            }
-
-            return allOf(futures);
         };
 
         // NB: all vv.update() calls must be made from the synchronous part of the method (not in thenCompose()/etc!).
         CompletableFuture<?> localPartsUpdateFuture = localPartsByTableIdVv.update(causalityToken,
                 (previous, throwable) -> inBusyLock(busyLock, () -> {
-                    return getOrCreatePartitionStorages(table, parts).thenApply(u -> {
-                        var newValue = new HashMap<>(previous);
+                    return assignments.thenCompose(newAssignments -> {

Review Comment:
   I agree. Fixed.



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

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

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


[GitHub] [ignite-3] denis-chudov commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -947,7 +971,8 @@ private void initDataNodesAndTriggerKeysInMetaStorage(
                             revision
                     );
                 } else if (res.getAsBoolean()) {
-                    LOG.debug("Update zones' dataNodes value [zoneId = {}, dataNodes = {}, revision = {}]", zoneId, dataNodes, revision);
+                    LOG.debug("Update zones' dataNodes value [zoneId = {}, dataNodes = {}, revision = {}]",

Review Comment:
   I think this should be on info level.



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -341,6 +341,7 @@ private PartialNode startPartialNode(
         SchemaManager schemaManager = new SchemaManager(registry, tablesConfig, metaStorageMgr);
 
         DistributionZoneManager distributionZoneManager = new DistributionZoneManager(
+                null,

Review Comment:
   I am sure that it would be more correct to inject `registry` here - as it is done for `TableManager`.



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java:
##########
@@ -124,6 +124,7 @@ void setUp() {
         zonesConfiguration = registry.getConfiguration(DistributionZonesConfiguration.KEY);
 
         distributionZoneManager = new DistributionZoneManager(
+                null,

Review Comment:
   It seems weird to me that we have `DistributionZoneCausalityDataNodesTest`, and `DistributionZoneManager#waitZoneProcessing` is never tested - neither there, nor in other places. And this injection of `null` assumes that this method will not work in derived classes. Is it correct that there are no tests for `DistributionZoneManager#waitZoneProcessing`?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -600,24 +600,25 @@ private CompletableFuture<?> onTableCreate(ConfigurationNotificationEvent<TableV
             CatalogTableDescriptor tableDescriptor = toTableDescriptor(ctx.newValue());
             CatalogZoneDescriptor zoneDescriptor = getZoneDescriptor(tableDescriptor.zoneId());
 
-            List<Set<Assignment>> assignments;
+            CompletableFuture<List<Set<Assignment>>> assignments;

Review Comment:
   It would be better to rename this variable to `assignmentsFuture`, as well as the parameter in methods.



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,717 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * The map which contains configuration changes which trigger zone's data nodes recalculation.
+     * zoneId -> (revision -> zoneConfiguration).
+     * TODO IGNITE-20050 Clean up this map.
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * The constructor.
+     *
+     * @param busyLock Busy lock to stop synchronously.
+     * @param msManager Meta Storage manager.
+     * @param vaultMgr Vault manager.
+     * @param zonesState Map with states for distribution zones.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public CausalityDataNodesEngine(
+            IgniteSpinBusyLock busyLock,
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.busyLock = busyLock;
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.distributionZoneManager = distributionZoneManager;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the exception:
+     * {@link IllegalArgumentException} if causalityToken or zoneId is not valid.
+     * {@link DistributionZoneNotFoundException} if the zone with the provided zoneId does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The data nodes for the zoneId.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+            // Get the latest configuration and configuration revision for a given causality token
+            Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+            if (zoneLastCfgEntry == null) {
+                // It means that the zone does not exist on a given causality token.
+                throw new DistributionZoneNotFoundException(zoneId);
+            }
+
+            long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+            ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+            String filter = zoneLastCfg.getFilter();
+
+            boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+            if (isZoneRemoved) {
+                // It means that the zone was removed on a given causality token.
+                throw new DistributionZoneNotFoundException(zoneId);
+            }
+
+            // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+            long lastScaleUpRevision = getRevisionsOfLastScaleUpEvent(causalityToken, zoneId);
+            long lastScaleDownRevision = getRevisionsOfLastScaleDownEvent(causalityToken, zoneId);
+
+            if (lastCfgRevision == versionedCfg.firstKey()
+                    && lastCfgRevision >= lastScaleUpRevision
+                    && lastCfgRevision >= lastScaleDownRevision
+            ) {
+                // It means that the zone was created but the data nodes value had not updated yet.
+                // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+                Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+                if (topologyEntry.value() == null) {

Review Comment:
   In general, it seems to be incorrect - we have working cluster and no nodes in logical topology. Please add TODO with IGNITE-20053 to process it, or even assertion, if it wouldn't cause the tests failure (most likely it shouldnt)



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -705,219 +705,230 @@ private CompletableFuture<?> onTableDelete(ConfigurationNotificationEvent<TableV
      */
     private CompletableFuture<?> createTablePartitionsLocally(
             long causalityToken,
-            List<Set<Assignment>> assignments,
+            CompletableFuture<List<Set<Assignment>>> assignments,
             int zoneId,
             TableImpl table
     ) {
         int tableId = table.tableId();
 
-        List<Set<Assignment>> newAssignments = assignments;
+        // Create new raft nodes according to new assignments.
+        Supplier<CompletableFuture<Void>> updateAssignmentsClosure = () -> {
+            return assignments.thenCompose(newAssignments -> {
+                // Empty assignments might be a valid case if tables are created from within cluster init HOCON
+                // configuration, which is not supported now.
+                assert newAssignments != null : IgniteStringFormatter.format("Table [id={}] has empty assignments.", tableId);
+
+                int partitions = newAssignments.size();
 
-        // Empty assignments might be a valid case if tables are created from within cluster init HOCON
-        // configuration, which is not supported now.
-        assert newAssignments != null : IgniteStringFormatter.format("Table [id={}] has empty assignments.", tableId);
+                CompletableFuture<?>[] futures = new CompletableFuture<?>[partitions];
 
-        int partitions = newAssignments.size();
+                // TODO: https://issues.apache.org/jira/browse/IGNITE-19713 Process assignments and set partitions only for assigned partitions.
+                PartitionSet parts = new BitSetPartitionSet();
 
-        CompletableFuture<?>[] futures = new CompletableFuture<?>[partitions];
+                for (int i = 0; i < futures.length; i++) {
+                    futures[i] = new CompletableFuture<>();
 
-        // TODO: https://issues.apache.org/jira/browse/IGNITE-19713 Process assignments and set partitions only for assigned partitions.
-        PartitionSet parts = new BitSetPartitionSet();
+                    parts.set(i);
+                }
 
-        for (int i = 0; i < futures.length; i++) {
-            futures[i] = new CompletableFuture<>();
+                String localMemberName = localNode().name();
 
-            parts.set(i);
-        }
+                for (int i = 0; i < partitions; i++) {
+                    int partId = i;
 
-        String localMemberName = localNode().name();
+                    Set<Assignment> newPartAssignment = newAssignments.get(partId);
 
-        // Create new raft nodes according to new assignments.
-        Supplier<CompletableFuture<Void>> updateAssignmentsClosure = () -> {
-            for (int i = 0; i < partitions; i++) {
-                int partId = i;
+                    InternalTable internalTbl = table.internalTable();
 
-                Set<Assignment> newPartAssignment = newAssignments.get(partId);
+                    Assignment localMemberAssignment = newPartAssignment.stream()
+                            .filter(a -> a.consistentId().equals(localMemberName))
+                            .findAny()
+                            .orElse(null);
 
-                InternalTable internalTbl = table.internalTable();
+                    PeersAndLearners newConfiguration = configurationFromAssignments(newPartAssignment);
 
-                Assignment localMemberAssignment = newPartAssignment.stream()
-                        .filter(a -> a.consistentId().equals(localMemberName))
-                        .findAny()
-                        .orElse(null);
+                    TablePartitionId replicaGrpId = new TablePartitionId(tableId, partId);
 
-                PeersAndLearners newConfiguration = configurationFromAssignments(newPartAssignment);
+                    placementDriver.updateAssignment(replicaGrpId, newConfiguration.peers().stream().map(Peer::consistentId)
+                            .collect(toList()));
 
-                TablePartitionId replicaGrpId = new TablePartitionId(tableId, partId);
+                    var safeTimeTracker = new PendingComparableValuesTracker<HybridTimestamp, Void>(
+                            new HybridTimestamp(1, 0)
+                    );
+                    var storageIndexTracker = new PendingComparableValuesTracker<Long, Void>(0L);
 
-                placementDriver.updateAssignment(replicaGrpId, newConfiguration.peers().stream().map(Peer::consistentId)
-                        .collect(toList()));
+                    ((InternalTableImpl) internalTbl).updatePartitionTrackers(partId, safeTimeTracker, storageIndexTracker);
 
-                var safeTimeTracker = new PendingComparableValuesTracker<HybridTimestamp, Void>(
-                        new HybridTimestamp(1, 0)
-                );
-                var storageIndexTracker = new PendingComparableValuesTracker<Long, Void>(0L);
+                    PartitionStorages partitionStorages = getPartitionStorages(table, partId);
 
-                ((InternalTableImpl) internalTbl).updatePartitionTrackers(partId, safeTimeTracker, storageIndexTracker);
+                    PartitionDataStorage partitionDataStorage = partitionDataStorage(partitionStorages.getMvPartitionStorage(),
+                            internalTbl, partId);
 
-                PartitionStorages partitionStorages = getPartitionStorages(table, partId);
+                    PartitionUpdateHandlers partitionUpdateHandlers = createPartitionUpdateHandlers(
+                            partId,
+                            partitionDataStorage,
+                            table,
+                            safeTimeTracker
+                    );
 
-                PartitionDataStorage partitionDataStorage = partitionDataStorage(partitionStorages.getMvPartitionStorage(),
-                        internalTbl, partId);
+                    mvGc.addStorage(replicaGrpId, partitionUpdateHandlers.gcUpdateHandler);
 
-                PartitionUpdateHandlers partitionUpdateHandlers = createPartitionUpdateHandlers(
-                        partId,
-                        partitionDataStorage,
-                        table,
-                        safeTimeTracker
-                );
+                    CompletableFuture<Boolean> startGroupFut;
 
-                mvGc.addStorage(replicaGrpId, partitionUpdateHandlers.gcUpdateHandler);
+                    // start new nodes, only if it is table creation, other cases will be covered by rebalance logic
+                    if (localMemberAssignment != null) {
+                        CompletableFuture<Boolean> shouldStartGroupFut;
 
-                CompletableFuture<Boolean> startGroupFut;
+                        // If Raft is running in in-memory mode or the PDS has been cleared, we need to remove the current node
+                        // from the Raft group in order to avoid the double vote problem.
+                        // <MUTED> See https://issues.apache.org/jira/browse/IGNITE-16668 for details.
+                        // TODO: https://issues.apache.org/jira/browse/IGNITE-19046 Restore "|| !hasData"
+                        if (internalTbl.storage().isVolatile()) {
+                            shouldStartGroupFut = queryDataNodesCount(tableId, partId, newConfiguration.peers())
+                                    .thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
 
-                // start new nodes, only if it is table creation, other cases will be covered by rebalance logic
-                if (localMemberAssignment != null) {
-                    CompletableFuture<Boolean> shouldStartGroupFut;
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
 
-                    // If Raft is running in in-memory mode or the PDS has been cleared, we need to remove the current node
-                    // from the Raft group in order to avoid the double vote problem.
-                    // <MUTED> See https://issues.apache.org/jira/browse/IGNITE-16668 for details.
-                    // TODO: https://issues.apache.org/jira/browse/IGNITE-19046 Restore "|| !hasData"
-                    if (internalTbl.storage().isVolatile()) {
-                        shouldStartGroupFut = queryDataNodesCount(tableId, partId, newConfiguration.peers()).thenApply(dataNodesCount -> {
-                            boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (newConfiguration.peers().size() / 2) + 1;
 
-                            if (fullPartitionRestart) {
-                                return true;
-                            }
+                                        if (majorityAvailable) {
+                                            RebalanceUtil.startPeerRemoval(replicaGrpId, localMemberAssignment, metaStorageMgr);
 
-                            boolean majorityAvailable = dataNodesCount >= (newConfiguration.peers().size() / 2) + 1;
+                                            return false;
+                                        } else {
+                                            // No majority and not a full partition restart - need to restart nodes
+                                            // with current partition.
+                                            String msg = "Unable to start partition " + partId + ". Majority not available.";
 
-                            if (majorityAvailable) {
-                                RebalanceUtil.startPeerRemoval(replicaGrpId, localMemberAssignment, metaStorageMgr);
+                                            throw new IgniteInternalException(msg);
+                                        }
+                                    });
+                        } else {
+                            shouldStartGroupFut = completedFuture(true);
+                        }
 
+                        startGroupFut = shouldStartGroupFut.thenApplyAsync(startGroup -> inBusyLock(busyLock, () -> {
+                            if (!startGroup) {
                                 return false;
-                            } else {
-                                // No majority and not a full partition restart - need to restart nodes
-                                // with current partition.
-                                String msg = "Unable to start partition " + partId + ". Majority not available.";
+                            }
+                            TxStateStorage txStatePartitionStorage = partitionStorages.getTxStateStorage();
+
+                            RaftGroupOptions groupOptions = groupOptionsForPartition(
+                                    internalTbl.storage(),
+                                    internalTbl.txStateStorage(),
+                                    partitionKey(internalTbl, partId),
+                                    partitionUpdateHandlers
+                            );
+
+                            Peer serverPeer = newConfiguration.peer(localMemberName);
+
+                            var raftNodeId = new RaftNodeId(replicaGrpId, serverPeer);
+
+                            try {
+                                // TODO: use RaftManager interface, see https://issues.apache.org/jira/browse/IGNITE-18273
+                                ((Loza) raftMgr).startRaftGroupNode(
+                                        raftNodeId,
+                                        newConfiguration,
+                                        new PartitionListener(
+                                                partitionDataStorage,
+                                                partitionUpdateHandlers.storageUpdateHandler,
+                                                txStatePartitionStorage,
+                                                safeTimeTracker,
+                                                storageIndexTracker
+                                        ),
+                                        new RebalanceRaftGroupEventsListener(
+                                                metaStorageMgr,
+                                                replicaGrpId,
+                                                busyLock,
+                                                createPartitionMover(internalTbl, partId),
+                                                this::calculateAssignments,
+                                                rebalanceScheduler
+                                        ),
+                                        groupOptions
+                                );
 
-                                throw new IgniteInternalException(msg);
+                                return true;
+                            } catch (NodeStoppingException ex) {
+                                throw new CompletionException(ex);
                             }
-                        });
+                        }), ioExecutor);
                     } else {
-                        shouldStartGroupFut = completedFuture(true);
+                        startGroupFut = completedFuture(false);
                     }
 
-                    startGroupFut = shouldStartGroupFut.thenApplyAsync(startGroup -> inBusyLock(busyLock, () -> {
-                        if (!startGroup) {
-                            return false;
-                        }
-                        TxStateStorage txStatePartitionStorage = partitionStorages.getTxStateStorage();
-
-                        RaftGroupOptions groupOptions = groupOptionsForPartition(
-                                internalTbl.storage(),
-                                internalTbl.txStateStorage(),
-                                partitionKey(internalTbl, partId),
-                                partitionUpdateHandlers
-                        );
+                    startGroupFut
+                            .thenComposeAsync(v -> inBusyLock(busyLock, () -> {
+                                try {
+                                    //TODO IGNITE-19614 This procedure takes 10 seconds if there's no majority online.
+                                    return raftMgr.startRaftGroupService(replicaGrpId, newConfiguration, raftGroupServiceFactory);
+                                } catch (NodeStoppingException ex) {
+                                    return failedFuture(ex);
+                                }
+                            }), ioExecutor)
+                            .thenAcceptAsync(updatedRaftGroupService -> inBusyLock(busyLock, () -> {
+                                ((InternalTableImpl) internalTbl).updateInternalTableRaftGroupService(partId, updatedRaftGroupService);
 
-                        Peer serverPeer = newConfiguration.peer(localMemberName);
+                                boolean startedRaftNode = startGroupFut.join();
+                                if (localMemberAssignment == null || !startedRaftNode) {
+                                    return;
+                                }
 
-                        var raftNodeId = new RaftNodeId(replicaGrpId, serverPeer);
+                                MvPartitionStorage partitionStorage = partitionStorages.getMvPartitionStorage();
+                                TxStateStorage txStateStorage = partitionStorages.getTxStateStorage();
 
-                        try {
-                            // TODO: use RaftManager interface, see https://issues.apache.org/jira/browse/IGNITE-18273
-                            ((Loza) raftMgr).startRaftGroupNode(
-                                    raftNodeId,
-                                    newConfiguration,
-                                    new PartitionListener(
-                                            partitionDataStorage,
-                                            partitionUpdateHandlers.storageUpdateHandler,
-                                            txStatePartitionStorage,
-                                            safeTimeTracker,
-                                            storageIndexTracker
-                                    ),
-                                    new RebalanceRaftGroupEventsListener(
-                                            metaStorageMgr,
+                                try {
+                                    startReplicaWithNewListener(
                                             replicaGrpId,
-                                            busyLock,
-                                            createPartitionMover(internalTbl, partId),
-                                            this::calculateAssignments,
-                                            rebalanceScheduler
-                                    ),
-                                    groupOptions
-                            );
+                                            table,
+                                            safeTimeTracker,
+                                            storageIndexTracker,
+                                            partitionStorage,
+                                            txStateStorage,
+                                            partitionUpdateHandlers,
+                                            updatedRaftGroupService,
+                                            schemaManager.schemaRegistry(causalityToken, tableId)
+                                    );
+                                } catch (NodeStoppingException ex) {
+                                    throw new AssertionError("Loza was stopped before Table manager", ex);
+                                }
+                            }), ioExecutor)
+                            .whenComplete((res, ex) -> {
+                                if (ex != null) {
+                                    LOG.warn("Unable to update raft groups on the node [tableId={}, partitionId={}]", ex, tableId, partId);
 
-                            return true;
-                        } catch (NodeStoppingException ex) {
-                            throw new CompletionException(ex);
-                        }
-                    }), ioExecutor);
-                } else {
-                    startGroupFut = completedFuture(false);
+                                    futures[partId].completeExceptionally(ex);
+                                } else {
+                                    futures[partId].complete(null);
+                                }
+                            });
                 }
 
-                startGroupFut
-                        .thenComposeAsync(v -> inBusyLock(busyLock, () -> {
-                            try {
-                                //TODO IGNITE-19614 This procedure takes 10 seconds if there's no majority online.
-                                return raftMgr.startRaftGroupService(replicaGrpId, newConfiguration, raftGroupServiceFactory);
-                            } catch (NodeStoppingException ex) {
-                                return failedFuture(ex);
-                            }
-                        }), ioExecutor)
-                        .thenAcceptAsync(updatedRaftGroupService -> inBusyLock(busyLock, () -> {
-                            ((InternalTableImpl) internalTbl).updateInternalTableRaftGroupService(partId, updatedRaftGroupService);
-
-                            boolean startedRaftNode = startGroupFut.join();
-                            if (localMemberAssignment == null || !startedRaftNode) {
-                                return;
-                            }
+                return allOf(futures);
+            });
 
-                            MvPartitionStorage partitionStorage = partitionStorages.getMvPartitionStorage();
-                            TxStateStorage txStateStorage = partitionStorages.getTxStateStorage();
 
-                            try {
-                                startReplicaWithNewListener(
-                                        replicaGrpId,
-                                        table,
-                                        safeTimeTracker,
-                                        storageIndexTracker,
-                                        partitionStorage,
-                                        txStateStorage,
-                                        partitionUpdateHandlers,
-                                        updatedRaftGroupService,
-                                        schemaManager.schemaRegistry(causalityToken, tableId)
-                                );
-                            } catch (NodeStoppingException ex) {
-                                throw new AssertionError("Loza was stopped before Table manager", ex);
-                            }
-                        }), ioExecutor)
-                        .whenComplete((res, ex) -> {
-                            if (ex != null) {
-                                LOG.warn("Unable to update raft groups on the node [tableId={}, partitionId={}]", ex, tableId, partId);
-
-                                futures[partId].completeExceptionally(ex);
-                            } else {
-                                futures[partId].complete(null);
-                            }
-                        });
-            }
-
-            return allOf(futures);
         };
 
         // NB: all vv.update() calls must be made from the synchronous part of the method (not in thenCompose()/etc!).
         CompletableFuture<?> localPartsUpdateFuture = localPartsByTableIdVv.update(causalityToken,
                 (previous, throwable) -> inBusyLock(busyLock, () -> {
-                    return getOrCreatePartitionStorages(table, parts).thenApply(u -> {
-                        var newValue = new HashMap<>(previous);
+                    return assignments.thenCompose(newAssignments -> {

Review Comment:
   Idea suggests to replace it with lambda - what do you think?



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            // It means that the zone does not exist on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        if (isZoneRemoved) {
+            // It means that the zone was removed on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            // It means that the zone was created but the data nodes value had not updated yet.
+            // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        LOG.info("+++++++ dataNodes zoneState " + zoneState);
+
+        ConcurrentSkipListMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        // The zoneState can be null if the zone was removed.
+        if (zoneState != null) {
+            subAugmentationMap = new ConcurrentSkipListMap<>(zoneState.topologyAugmentationMap()
+                    .headMap(causalityToken, true));
+        }
+
+        // Search the revisions of zoneScaleUpChangeTriggerKey and zoneScaleDownChangeTriggerKey with value greater or equals
+        // to expected one.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        // Choose the highest revision.
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        // Read data nodes value from the meta storage on dataNodesRevision and associated trigger keys.
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        if (dataNodesEntry.value() == null) {
+            LOG.info("+++++++ dataNodes The zone was removed not idempotently");
+            // The zone was removed.
+            // In this case it is impossible to find out the data nodes value idempotently.
+            return emptySet();
+        }
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            // Update the data nodes set with pending data from augmentation map
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        // Apply the filter to get the final data nodes set.
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        // Iterate over zone configurations from newest to oldest.
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()
+                        && newerCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE
+                        && isScaleUpImmediate) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleDownRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleDown() != newerCfg.getDataNodesAutoAdjustScaleDown()
+                        && newerCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE
+                        && isScaleDownImmediate) {
+                    scaleDownRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleUpRevision == 0 && !olderCfg.getFilter().equals(newerCfg.getFilter())) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+            }
+
+            if ((scaleUpRevision > 0) && (scaleDownRevision > 0)) {
+                break;
+            }
+
+            entryNewerCfg = entryOlderCfg;
+        }
+
+        // The case when there is only one configuration in the history.
+        if (entryNewerCfg != null) {
+            if (scaleUpRevision == 0) {
+                scaleUpRevision = entryNewerCfg.getKey();
+            }
+
+            if (scaleDownRevision == 0) {
+                scaleDownRevision = entryNewerCfg.getKey();
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpRevision, scaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest topology event with added nodes and with removed nodes when the zone have
+     * immediate scale up and scale down timers.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownTopologyRevisions(long causalityToken, int zoneId) {
+        Set<NodeWithAttributes> newerLogicalTopology;
+
+        long newerTopologyRevision;
+
+        Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), causalityToken);
+
+        long scaleUpTopologyRevision = 0;
+        long scaleDownTopologyRevision = 0;
+
+
+        if (!topologyEntry.empty()) {
+            byte[] newerLogicalTopologyBytes = topologyEntry.value();
+
+            newerLogicalTopology = fromBytes(newerLogicalTopologyBytes);
+
+            newerTopologyRevision = topologyEntry.revision();
+
+            while ((scaleUpTopologyRevision == 0) || (scaleDownTopologyRevision == 0)) {
+                topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), newerTopologyRevision - 1);
+
+                Set<NodeWithAttributes> olderLogicalTopology;
+
+                if (topologyEntry.empty()) {
+                    // If older topology is empty then it means that each topology changes were iterated
+                    // so use empty set to compare it with the first topology.
+                    olderLogicalTopology = emptySet();
+                } else {
+                    byte[] olderLogicalTopologyBytes = topologyEntry.value();
+
+                    olderLogicalTopology = fromBytes(olderLogicalTopologyBytes);
+                }
+
+                Set<NodeWithAttributes> finalNewerLogicalTopology = newerLogicalTopology;
+
+                Set<Node> removedNodes =
+                        olderLogicalTopology.stream()
+                                .filter(node -> !finalNewerLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Set<Node> addedNodes =
+                        newerLogicalTopology.stream()
+                                .filter(node -> !olderLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Map.Entry<Long, ZoneConfiguration> zoneConfigurationEntry = zonesVersionedCfg.get(zoneId)
+                        .floorEntry(newerTopologyRevision);
+
+                if (zoneConfigurationEntry == null) {
+                    break;
+                }
+
+                ZoneConfiguration zoneCfg = zoneConfigurationEntry.getValue();
+
+                if (scaleUpTopologyRevision == 0
+                        && !addedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE) {
+                    scaleUpTopologyRevision = newerTopologyRevision;
+                }
+
+                if (scaleDownTopologyRevision == 0
+                        && !removedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE) {
+                    scaleDownTopologyRevision = newerTopologyRevision;
+                }
+
+                newerLogicalTopology = olderLogicalTopology;
+
+                newerTopologyRevision = topologyEntry.revision();
+
+                if (topologyEntry.empty()) {
+                    break;
+                }
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpTopologyRevision, scaleDownTopologyRevision);
+    }
+
+    /**
+     * Search a value of zoneScaleUpChangeTriggerKey/zoneScaleDownChangeTriggerKey which equals or greater than scaleRevision.
+     * It iterates over the entries in the local meta storage. If there is an entry with a value equals to or greater than
+     * the scaleRevision, then it returns the revision of this entry. If there is no such entry then it returns zero.
+     *
+     * @param scaleRevision Scale revision.
+     * @param zoneId Zone id.
+     * @param triggerKey Trigger key.
+     * @return Revision.
+     */
+    private long searchTriggerKey(Long scaleRevision, int zoneId, ByteArray triggerKey) {
+        System.out.println("searchTriggerKey " + scaleRevision + " " + zoneId);
+
+        Entry lastEntry = msManager.getLocally(triggerKey, Long.MAX_VALUE);
+
+        long upperRevision = max(lastEntry.revision(), scaleRevision);
+
+        // Gets old entries from storage to check if the expected value was handled before watch listener was registered.
+        List<Entry> entryList = msManager.getLocally(triggerKey.bytes(), scaleRevision, upperRevision);
+
+        for (Entry entry : entryList) {
+
+            // scaleRevision is null if the zone was removed.
+            if (entry.value() == null) {
+                return entry.revision();
+            } else {
+                long entryScaleRevision = bytesToLong(entry.value());
+
+                if (entryScaleRevision >= scaleRevision) {
+                    return entry.revision();
+                }
+            }
+        }
+
+        return 0;
+    }
+
+    /**
+     * causalityOnUpdateScaleUp.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleUp newScaleUp.
+     */
+    public void causalityOnUpdateScaleUp(long revision, int zoneId, int newScaleUp) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleUp(newScaleUp);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * causalityOnUpdateScaleDown.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleDown newScaleDown.
+     */
+    public void causalityOnUpdateScaleDown(long revision, int zoneId, int newScaleDown) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleDown(newScaleDown);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * onUpdateFilter.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param filter filter.
+     */
+    public void onUpdateFilter(long revision, int zoneId, String filter) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setFilter(filter);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * onCreateOrRestoreZoneState.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param zoneCreation zoneCreation.
+     * @param zone zone.
+     */
+    public void onCreateOrRestoreZoneState(long revision, int zoneId, boolean zoneCreation, DistributionZoneView zone) {
+        if (zoneCreation) {
+            ZoneConfiguration zoneConfiguration = new ZoneConfiguration(
+                    false,
+                    zone.dataNodesAutoAdjustScaleUp(),
+                    zone.dataNodesAutoAdjustScaleDown(),
+                    zone.filter()
+            );
+
+            ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = new ConcurrentSkipListMap<>();
+
+            versionedCfg.put(revision, zoneConfiguration);
+
+            zonesVersionedCfg.put(zoneId, versionedCfg);
+
+            vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+
+        } else {
+            VaultEntry versionedCfgEntry = vaultMgr.get(zoneVersionedConfigurationKey(zoneId)).join();
+
+            if (versionedCfgEntry != null) {
+                zonesVersionedCfg.put(zoneId, fromBytes(versionedCfgEntry.value()));
+            }
+        }
+    }
+
+    /**
+     * onDelete.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     */
+    public void onDelete(long revision, int zoneId) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setIsRemoved(true);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * Class stores zone configuration parameters. Changing of these parameters can trigger a data nodes recalculation.
+     */
+    public static class ZoneConfiguration implements Serializable {

Review Comment:
   ZoneConfiguration is more simple than DistributionZoneView.



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            // It means that the zone does not exist on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        if (isZoneRemoved) {
+            // It means that the zone was removed on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            // It means that the zone was created but the data nodes value had not updated yet.
+            // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        LOG.info("+++++++ dataNodes zoneState " + zoneState);
+
+        ConcurrentSkipListMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        // The zoneState can be null if the zone was removed.
+        if (zoneState != null) {
+            subAugmentationMap = new ConcurrentSkipListMap<>(zoneState.topologyAugmentationMap()
+                    .headMap(causalityToken, true));
+        }
+
+        // Search the revisions of zoneScaleUpChangeTriggerKey and zoneScaleDownChangeTriggerKey with value greater or equals
+        // to expected one.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        // Choose the highest revision.
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        // Read data nodes value from the meta storage on dataNodesRevision and associated trigger keys.
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        if (dataNodesEntry.value() == null) {
+            LOG.info("+++++++ dataNodes The zone was removed not idempotently");
+            // The zone was removed.
+            // In this case it is impossible to find out the data nodes value idempotently.
+            return emptySet();
+        }
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            // Update the data nodes set with pending data from augmentation map
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        // Apply the filter to get the final data nodes set.
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        // Iterate over zone configurations from newest to oldest.
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()
+                        && newerCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE
+                        && isScaleUpImmediate) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleDownRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleDown() != newerCfg.getDataNodesAutoAdjustScaleDown()
+                        && newerCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE
+                        && isScaleDownImmediate) {
+                    scaleDownRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleUpRevision == 0 && !olderCfg.getFilter().equals(newerCfg.getFilter())) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+            }
+
+            if ((scaleUpRevision > 0) && (scaleDownRevision > 0)) {
+                break;
+            }
+
+            entryNewerCfg = entryOlderCfg;
+        }
+
+        // The case when there is only one configuration in the history.
+        if (entryNewerCfg != null) {
+            if (scaleUpRevision == 0) {
+                scaleUpRevision = entryNewerCfg.getKey();
+            }
+
+            if (scaleDownRevision == 0) {
+                scaleDownRevision = entryNewerCfg.getKey();
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpRevision, scaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest topology event with added nodes and with removed nodes when the zone have
+     * immediate scale up and scale down timers.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownTopologyRevisions(long causalityToken, int zoneId) {
+        Set<NodeWithAttributes> newerLogicalTopology;
+
+        long newerTopologyRevision;
+
+        Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), causalityToken);
+
+        long scaleUpTopologyRevision = 0;
+        long scaleDownTopologyRevision = 0;
+
+
+        if (!topologyEntry.empty()) {
+            byte[] newerLogicalTopologyBytes = topologyEntry.value();
+
+            newerLogicalTopology = fromBytes(newerLogicalTopologyBytes);
+
+            newerTopologyRevision = topologyEntry.revision();
+
+            while ((scaleUpTopologyRevision == 0) || (scaleDownTopologyRevision == 0)) {
+                topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), newerTopologyRevision - 1);
+
+                Set<NodeWithAttributes> olderLogicalTopology;
+
+                if (topologyEntry.empty()) {
+                    // If older topology is empty then it means that each topology changes were iterated
+                    // so use empty set to compare it with the first topology.
+                    olderLogicalTopology = emptySet();
+                } else {
+                    byte[] olderLogicalTopologyBytes = topologyEntry.value();
+
+                    olderLogicalTopology = fromBytes(olderLogicalTopologyBytes);
+                }
+
+                Set<NodeWithAttributes> finalNewerLogicalTopology = newerLogicalTopology;
+
+                Set<Node> removedNodes =
+                        olderLogicalTopology.stream()
+                                .filter(node -> !finalNewerLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Set<Node> addedNodes =
+                        newerLogicalTopology.stream()
+                                .filter(node -> !olderLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Map.Entry<Long, ZoneConfiguration> zoneConfigurationEntry = zonesVersionedCfg.get(zoneId)
+                        .floorEntry(newerTopologyRevision);
+
+                if (zoneConfigurationEntry == null) {
+                    break;
+                }
+
+                ZoneConfiguration zoneCfg = zoneConfigurationEntry.getValue();
+
+                if (scaleUpTopologyRevision == 0
+                        && !addedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE) {
+                    scaleUpTopologyRevision = newerTopologyRevision;
+                }
+
+                if (scaleDownTopologyRevision == 0
+                        && !removedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE) {
+                    scaleDownTopologyRevision = newerTopologyRevision;
+                }
+
+                newerLogicalTopology = olderLogicalTopology;
+
+                newerTopologyRevision = topologyEntry.revision();
+
+                if (topologyEntry.empty()) {
+                    break;
+                }
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpTopologyRevision, scaleDownTopologyRevision);
+    }
+
+    /**
+     * Search a value of zoneScaleUpChangeTriggerKey/zoneScaleDownChangeTriggerKey which equals or greater than scaleRevision.
+     * It iterates over the entries in the local meta storage. If there is an entry with a value equals to or greater than
+     * the scaleRevision, then it returns the revision of this entry. If there is no such entry then it returns zero.
+     *
+     * @param scaleRevision Scale revision.
+     * @param zoneId Zone id.
+     * @param triggerKey Trigger key.
+     * @return Revision.
+     */
+    private long searchTriggerKey(Long scaleRevision, int zoneId, ByteArray triggerKey) {
+        System.out.println("searchTriggerKey " + scaleRevision + " " + zoneId);
+
+        Entry lastEntry = msManager.getLocally(triggerKey, Long.MAX_VALUE);
+
+        long upperRevision = max(lastEntry.revision(), scaleRevision);
+
+        // Gets old entries from storage to check if the expected value was handled before watch listener was registered.
+        List<Entry> entryList = msManager.getLocally(triggerKey.bytes(), scaleRevision, upperRevision);
+
+        for (Entry entry : entryList) {
+
+            // scaleRevision is null if the zone was removed.
+            if (entry.value() == null) {
+                return entry.revision();
+            } else {
+                long entryScaleRevision = bytesToLong(entry.value());
+
+                if (entryScaleRevision >= scaleRevision) {
+                    return entry.revision();
+                }
+            }
+        }
+
+        return 0;
+    }
+
+    /**
+     * causalityOnUpdateScaleUp.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleUp newScaleUp.
+     */
+    public void causalityOnUpdateScaleUp(long revision, int zoneId, int newScaleUp) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleUp(newScaleUp);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * causalityOnUpdateScaleDown.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleDown newScaleDown.
+     */
+    public void causalityOnUpdateScaleDown(long revision, int zoneId, int newScaleDown) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleDown(newScaleDown);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * onUpdateFilter.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param filter filter.
+     */
+    public void onUpdateFilter(long revision, int zoneId, String filter) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setFilter(filter);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * onCreateOrRestoreZoneState.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param zoneCreation zoneCreation.
+     * @param zone zone.
+     */
+    public void onCreateOrRestoreZoneState(long revision, int zoneId, boolean zoneCreation, DistributionZoneView zone) {

Review Comment:
   Fixed.



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

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

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


[GitHub] [ignite-3] alievmirza commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,

Review Comment:
   why do we need to propagate `zoneState` and `nodesAttriburtes`, if we could just pass here `distributionZoneManager` itself and access these fields? 



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            // It means that the zone does not exist on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        if (isZoneRemoved) {
+            // It means that the zone was removed on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            // It means that the zone was created but the data nodes value had not updated yet.
+            // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        LOG.info("+++++++ dataNodes zoneState " + zoneState);
+
+        ConcurrentSkipListMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        // The zoneState can be null if the zone was removed.
+        if (zoneState != null) {
+            subAugmentationMap = new ConcurrentSkipListMap<>(zoneState.topologyAugmentationMap()
+                    .headMap(causalityToken, true));
+        }
+
+        // Search the revisions of zoneScaleUpChangeTriggerKey and zoneScaleDownChangeTriggerKey with value greater or equals
+        // to expected one.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        // Choose the highest revision.
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        // Read data nodes value from the meta storage on dataNodesRevision and associated trigger keys.
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        if (dataNodesEntry.value() == null) {
+            LOG.info("+++++++ dataNodes The zone was removed not idempotently");
+            // The zone was removed.
+            // In this case it is impossible to find out the data nodes value idempotently.
+            return emptySet();
+        }
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            // Update the data nodes set with pending data from augmentation map
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        // Apply the filter to get the final data nodes set.
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        // Iterate over zone configurations from newest to oldest.
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()
+                        && newerCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE
+                        && isScaleUpImmediate) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleDownRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleDown() != newerCfg.getDataNodesAutoAdjustScaleDown()
+                        && newerCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE
+                        && isScaleDownImmediate) {
+                    scaleDownRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleUpRevision == 0 && !olderCfg.getFilter().equals(newerCfg.getFilter())) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+            }
+
+            if ((scaleUpRevision > 0) && (scaleDownRevision > 0)) {
+                break;
+            }
+
+            entryNewerCfg = entryOlderCfg;
+        }
+
+        // The case when there is only one configuration in the history.
+        if (entryNewerCfg != null) {
+            if (scaleUpRevision == 0) {
+                scaleUpRevision = entryNewerCfg.getKey();
+            }
+
+            if (scaleDownRevision == 0) {
+                scaleDownRevision = entryNewerCfg.getKey();
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpRevision, scaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest topology event with added nodes and with removed nodes when the zone have
+     * immediate scale up and scale down timers.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownTopologyRevisions(long causalityToken, int zoneId) {
+        Set<NodeWithAttributes> newerLogicalTopology;
+
+        long newerTopologyRevision;
+
+        Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), causalityToken);
+
+        long scaleUpTopologyRevision = 0;
+        long scaleDownTopologyRevision = 0;
+
+
+        if (!topologyEntry.empty()) {
+            byte[] newerLogicalTopologyBytes = topologyEntry.value();
+
+            newerLogicalTopology = fromBytes(newerLogicalTopologyBytes);
+
+            newerTopologyRevision = topologyEntry.revision();
+
+            while ((scaleUpTopologyRevision == 0) || (scaleDownTopologyRevision == 0)) {
+                topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), newerTopologyRevision - 1);
+
+                Set<NodeWithAttributes> olderLogicalTopology;
+
+                if (topologyEntry.empty()) {
+                    // If older topology is empty then it means that each topology changes were iterated
+                    // so use empty set to compare it with the first topology.
+                    olderLogicalTopology = emptySet();
+                } else {
+                    byte[] olderLogicalTopologyBytes = topologyEntry.value();
+
+                    olderLogicalTopology = fromBytes(olderLogicalTopologyBytes);
+                }
+
+                Set<NodeWithAttributes> finalNewerLogicalTopology = newerLogicalTopology;
+
+                Set<Node> removedNodes =
+                        olderLogicalTopology.stream()
+                                .filter(node -> !finalNewerLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Set<Node> addedNodes =
+                        newerLogicalTopology.stream()
+                                .filter(node -> !olderLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Map.Entry<Long, ZoneConfiguration> zoneConfigurationEntry = zonesVersionedCfg.get(zoneId)
+                        .floorEntry(newerTopologyRevision);
+
+                if (zoneConfigurationEntry == null) {
+                    break;
+                }
+
+                ZoneConfiguration zoneCfg = zoneConfigurationEntry.getValue();
+
+                if (scaleUpTopologyRevision == 0
+                        && !addedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE) {
+                    scaleUpTopologyRevision = newerTopologyRevision;
+                }
+
+                if (scaleDownTopologyRevision == 0
+                        && !removedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE) {
+                    scaleDownTopologyRevision = newerTopologyRevision;
+                }
+
+                newerLogicalTopology = olderLogicalTopology;
+
+                newerTopologyRevision = topologyEntry.revision();
+
+                if (topologyEntry.empty()) {
+                    break;
+                }
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpTopologyRevision, scaleDownTopologyRevision);
+    }
+
+    /**
+     * Search a value of zoneScaleUpChangeTriggerKey/zoneScaleDownChangeTriggerKey which equals or greater than scaleRevision.
+     * It iterates over the entries in the local meta storage. If there is an entry with a value equals to or greater than
+     * the scaleRevision, then it returns the revision of this entry. If there is no such entry then it returns zero.
+     *
+     * @param scaleRevision Scale revision.
+     * @param zoneId Zone id.
+     * @param triggerKey Trigger key.
+     * @return Revision.
+     */
+    private long searchTriggerKey(Long scaleRevision, int zoneId, ByteArray triggerKey) {
+        System.out.println("searchTriggerKey " + scaleRevision + " " + zoneId);
+
+        Entry lastEntry = msManager.getLocally(triggerKey, Long.MAX_VALUE);
+
+        long upperRevision = max(lastEntry.revision(), scaleRevision);
+
+        // Gets old entries from storage to check if the expected value was handled before watch listener was registered.
+        List<Entry> entryList = msManager.getLocally(triggerKey.bytes(), scaleRevision, upperRevision);
+
+        for (Entry entry : entryList) {
+
+            // scaleRevision is null if the zone was removed.
+            if (entry.value() == null) {
+                return entry.revision();
+            } else {
+                long entryScaleRevision = bytesToLong(entry.value());
+
+                if (entryScaleRevision >= scaleRevision) {
+                    return entry.revision();
+                }
+            }
+        }
+
+        return 0;
+    }
+
+    /**
+     * causalityOnUpdateScaleUp.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleUp newScaleUp.
+     */
+    public void causalityOnUpdateScaleUp(long revision, int zoneId, int newScaleUp) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleUp(newScaleUp);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * causalityOnUpdateScaleDown.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleDown newScaleDown.
+     */
+    public void causalityOnUpdateScaleDown(long revision, int zoneId, int newScaleDown) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleDown(newScaleDown);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * onUpdateFilter.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param filter filter.
+     */
+    public void onUpdateFilter(long revision, int zoneId, String filter) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setFilter(filter);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * onCreateOrRestoreZoneState.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param zoneCreation zoneCreation.
+     * @param zone zone.
+     */
+    public void onCreateOrRestoreZoneState(long revision, int zoneId, boolean zoneCreation, DistributionZoneView zone) {

Review Comment:
   `zoneId` is redundant, we could retrieve it from `DistributionZoneView`



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            // It means that the zone does not exist on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        if (isZoneRemoved) {
+            // It means that the zone was removed on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            // It means that the zone was created but the data nodes value had not updated yet.
+            // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        LOG.info("+++++++ dataNodes zoneState " + zoneState);
+
+        ConcurrentSkipListMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        // The zoneState can be null if the zone was removed.
+        if (zoneState != null) {
+            subAugmentationMap = new ConcurrentSkipListMap<>(zoneState.topologyAugmentationMap()
+                    .headMap(causalityToken, true));
+        }
+
+        // Search the revisions of zoneScaleUpChangeTriggerKey and zoneScaleDownChangeTriggerKey with value greater or equals
+        // to expected one.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        // Choose the highest revision.
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        // Read data nodes value from the meta storage on dataNodesRevision and associated trigger keys.
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        if (dataNodesEntry.value() == null) {
+            LOG.info("+++++++ dataNodes The zone was removed not idempotently");
+            // The zone was removed.
+            // In this case it is impossible to find out the data nodes value idempotently.
+            return emptySet();
+        }
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            // Update the data nodes set with pending data from augmentation map
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        // Apply the filter to get the final data nodes set.
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        // Iterate over zone configurations from newest to oldest.
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()

Review Comment:
   this could be simplified, you check for `isScaleUpImmediate` twice



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            // It means that the zone does not exist on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        if (isZoneRemoved) {
+            // It means that the zone was removed on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            // It means that the zone was created but the data nodes value had not updated yet.
+            // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        LOG.info("+++++++ dataNodes zoneState " + zoneState);
+
+        ConcurrentSkipListMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        // The zoneState can be null if the zone was removed.
+        if (zoneState != null) {
+            subAugmentationMap = new ConcurrentSkipListMap<>(zoneState.topologyAugmentationMap()
+                    .headMap(causalityToken, true));
+        }
+
+        // Search the revisions of zoneScaleUpChangeTriggerKey and zoneScaleDownChangeTriggerKey with value greater or equals
+        // to expected one.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        // Choose the highest revision.
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        // Read data nodes value from the meta storage on dataNodesRevision and associated trigger keys.
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        if (dataNodesEntry.value() == null) {
+            LOG.info("+++++++ dataNodes The zone was removed not idempotently");
+            // The zone was removed.
+            // In this case it is impossible to find out the data nodes value idempotently.
+            return emptySet();
+        }
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            // Update the data nodes set with pending data from augmentation map
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        // Apply the filter to get the final data nodes set.
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(

Review Comment:
   Please, do not use `IgniteBiTuple`, let's have two methods to retrieve revisions 



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            // It means that the zone does not exist on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        if (isZoneRemoved) {
+            // It means that the zone was removed on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            // It means that the zone was created but the data nodes value had not updated yet.
+            // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        LOG.info("+++++++ dataNodes zoneState " + zoneState);
+
+        ConcurrentSkipListMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        // The zoneState can be null if the zone was removed.
+        if (zoneState != null) {
+            subAugmentationMap = new ConcurrentSkipListMap<>(zoneState.topologyAugmentationMap()
+                    .headMap(causalityToken, true));
+        }
+
+        // Search the revisions of zoneScaleUpChangeTriggerKey and zoneScaleDownChangeTriggerKey with value greater or equals
+        // to expected one.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        // Choose the highest revision.
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        // Read data nodes value from the meta storage on dataNodesRevision and associated trigger keys.
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        if (dataNodesEntry.value() == null) {
+            LOG.info("+++++++ dataNodes The zone was removed not idempotently");
+            // The zone was removed.
+            // In this case it is impossible to find out the data nodes value idempotently.
+            return emptySet();
+        }
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            // Update the data nodes set with pending data from augmentation map
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        // Apply the filter to get the final data nodes set.
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        // Iterate over zone configurations from newest to oldest.
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()
+                        && newerCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE
+                        && isScaleUpImmediate) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleDownRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleDown() != newerCfg.getDataNodesAutoAdjustScaleDown()
+                        && newerCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE
+                        && isScaleDownImmediate) {
+                    scaleDownRevision = entryNewerCfg.getKey();
+                }
+
+                if (scaleUpRevision == 0 && !olderCfg.getFilter().equals(newerCfg.getFilter())) {
+                    scaleUpRevision = entryNewerCfg.getKey();
+                }
+            }
+
+            if ((scaleUpRevision > 0) && (scaleDownRevision > 0)) {
+                break;
+            }
+
+            entryNewerCfg = entryOlderCfg;
+        }
+
+        // The case when there is only one configuration in the history.
+        if (entryNewerCfg != null) {
+            if (scaleUpRevision == 0) {
+                scaleUpRevision = entryNewerCfg.getKey();
+            }
+
+            if (scaleDownRevision == 0) {
+                scaleDownRevision = entryNewerCfg.getKey();
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpRevision, scaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest topology event with added nodes and with removed nodes when the zone have
+     * immediate scale up and scale down timers.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownTopologyRevisions(long causalityToken, int zoneId) {
+        Set<NodeWithAttributes> newerLogicalTopology;
+
+        long newerTopologyRevision;
+
+        Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), causalityToken);
+
+        long scaleUpTopologyRevision = 0;
+        long scaleDownTopologyRevision = 0;
+
+
+        if (!topologyEntry.empty()) {
+            byte[] newerLogicalTopologyBytes = topologyEntry.value();
+
+            newerLogicalTopology = fromBytes(newerLogicalTopologyBytes);
+
+            newerTopologyRevision = topologyEntry.revision();
+
+            while ((scaleUpTopologyRevision == 0) || (scaleDownTopologyRevision == 0)) {
+                topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), newerTopologyRevision - 1);
+
+                Set<NodeWithAttributes> olderLogicalTopology;
+
+                if (topologyEntry.empty()) {
+                    // If older topology is empty then it means that each topology changes were iterated
+                    // so use empty set to compare it with the first topology.
+                    olderLogicalTopology = emptySet();
+                } else {
+                    byte[] olderLogicalTopologyBytes = topologyEntry.value();
+
+                    olderLogicalTopology = fromBytes(olderLogicalTopologyBytes);
+                }
+
+                Set<NodeWithAttributes> finalNewerLogicalTopology = newerLogicalTopology;
+
+                Set<Node> removedNodes =
+                        olderLogicalTopology.stream()
+                                .filter(node -> !finalNewerLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Set<Node> addedNodes =
+                        newerLogicalTopology.stream()
+                                .filter(node -> !olderLogicalTopology.contains(node))
+                                .map(NodeWithAttributes::node)
+                                .collect(toSet());
+
+                Map.Entry<Long, ZoneConfiguration> zoneConfigurationEntry = zonesVersionedCfg.get(zoneId)
+                        .floorEntry(newerTopologyRevision);
+
+                if (zoneConfigurationEntry == null) {
+                    break;
+                }
+
+                ZoneConfiguration zoneCfg = zoneConfigurationEntry.getValue();
+
+                if (scaleUpTopologyRevision == 0
+                        && !addedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE) {
+                    scaleUpTopologyRevision = newerTopologyRevision;
+                }
+
+                if (scaleDownTopologyRevision == 0
+                        && !removedNodes.isEmpty()
+                        && zoneCfg.getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE) {
+                    scaleDownTopologyRevision = newerTopologyRevision;
+                }
+
+                newerLogicalTopology = olderLogicalTopology;
+
+                newerTopologyRevision = topologyEntry.revision();
+
+                if (topologyEntry.empty()) {
+                    break;
+                }
+            }
+        }
+
+        return new IgniteBiTuple<>(scaleUpTopologyRevision, scaleDownTopologyRevision);
+    }
+
+    /**
+     * Search a value of zoneScaleUpChangeTriggerKey/zoneScaleDownChangeTriggerKey which equals or greater than scaleRevision.
+     * It iterates over the entries in the local meta storage. If there is an entry with a value equals to or greater than
+     * the scaleRevision, then it returns the revision of this entry. If there is no such entry then it returns zero.
+     *
+     * @param scaleRevision Scale revision.
+     * @param zoneId Zone id.
+     * @param triggerKey Trigger key.
+     * @return Revision.
+     */
+    private long searchTriggerKey(Long scaleRevision, int zoneId, ByteArray triggerKey) {
+        System.out.println("searchTriggerKey " + scaleRevision + " " + zoneId);
+
+        Entry lastEntry = msManager.getLocally(triggerKey, Long.MAX_VALUE);
+
+        long upperRevision = max(lastEntry.revision(), scaleRevision);
+
+        // Gets old entries from storage to check if the expected value was handled before watch listener was registered.
+        List<Entry> entryList = msManager.getLocally(triggerKey.bytes(), scaleRevision, upperRevision);
+
+        for (Entry entry : entryList) {
+
+            // scaleRevision is null if the zone was removed.
+            if (entry.value() == null) {
+                return entry.revision();
+            } else {
+                long entryScaleRevision = bytesToLong(entry.value());
+
+                if (entryScaleRevision >= scaleRevision) {
+                    return entry.revision();
+                }
+            }
+        }
+
+        return 0;
+    }
+
+    /**
+     * causalityOnUpdateScaleUp.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleUp newScaleUp.
+     */
+    public void causalityOnUpdateScaleUp(long revision, int zoneId, int newScaleUp) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleUp(newScaleUp);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * causalityOnUpdateScaleDown.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param newScaleDown newScaleDown.
+     */
+    public void causalityOnUpdateScaleDown(long revision, int zoneId, int newScaleDown) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setDataNodesAutoAdjustScaleDown(newScaleDown);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * onUpdateFilter.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param filter filter.
+     */
+    public void onUpdateFilter(long revision, int zoneId, String filter) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setFilter(filter);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * onCreateOrRestoreZoneState.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     * @param zoneCreation zoneCreation.
+     * @param zone zone.
+     */
+    public void onCreateOrRestoreZoneState(long revision, int zoneId, boolean zoneCreation, DistributionZoneView zone) {
+        if (zoneCreation) {
+            ZoneConfiguration zoneConfiguration = new ZoneConfiguration(
+                    false,
+                    zone.dataNodesAutoAdjustScaleUp(),
+                    zone.dataNodesAutoAdjustScaleDown(),
+                    zone.filter()
+            );
+
+            ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = new ConcurrentSkipListMap<>();
+
+            versionedCfg.put(revision, zoneConfiguration);
+
+            zonesVersionedCfg.put(zoneId, versionedCfg);
+
+            vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+
+        } else {
+            VaultEntry versionedCfgEntry = vaultMgr.get(zoneVersionedConfigurationKey(zoneId)).join();
+
+            if (versionedCfgEntry != null) {
+                zonesVersionedCfg.put(zoneId, fromBytes(versionedCfgEntry.value()));
+            }
+        }
+    }
+
+    /**
+     * onDelete.
+     *
+     * @param revision revision.
+     * @param zoneId zoneId.
+     */
+    public void onDelete(long revision, int zoneId) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        ZoneConfiguration previousCfg = versionedCfg.floorEntry(revision).getValue();
+
+        ZoneConfiguration newCfg = new ZoneConfiguration(previousCfg).setIsRemoved(true);
+
+        versionedCfg.put(revision, newCfg);
+
+        vaultMgr.put(zoneVersionedConfigurationKey(zoneId), toBytes(versionedCfg)).join();
+    }
+
+    /**
+     * Class stores zone configuration parameters. Changing of these parameters can trigger a data nodes recalculation.
+     */
+    public static class ZoneConfiguration implements Serializable {

Review Comment:
   Why do we need a separate class, if we could use `DistributionZoneView` itself with all information? 



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,717 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * The map which contains configuration changes which trigger zone's data nodes recalculation.
+     * zoneId -> (revision -> zoneConfiguration).
+     * TODO IGNITE-20050 Clean up this map.
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * The constructor.
+     *
+     * @param busyLock Busy lock to stop synchronously.
+     * @param msManager Meta Storage manager.
+     * @param vaultMgr Vault manager.
+     * @param zonesState Map with states for distribution zones.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public CausalityDataNodesEngine(
+            IgniteSpinBusyLock busyLock,
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.busyLock = busyLock;
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.distributionZoneManager = distributionZoneManager;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the exception:
+     * {@link IllegalArgumentException} if causalityToken or zoneId is not valid.
+     * {@link DistributionZoneNotFoundException} if the zone with the provided zoneId does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The data nodes for the zoneId.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+            // Get the latest configuration and configuration revision for a given causality token
+            Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+            if (zoneLastCfgEntry == null) {
+                // It means that the zone does not exist on a given causality token.
+                throw new DistributionZoneNotFoundException(zoneId);
+            }
+
+            long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+            ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+            String filter = zoneLastCfg.getFilter();
+
+            boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+            if (isZoneRemoved) {
+                // It means that the zone was removed on a given causality token.
+                throw new DistributionZoneNotFoundException(zoneId);
+            }
+
+            // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+            long lastScaleUpRevision = getRevisionsOfLastScaleUpEvent(causalityToken, zoneId);
+            long lastScaleDownRevision = getRevisionsOfLastScaleDownEvent(causalityToken, zoneId);
+
+            if (lastCfgRevision == versionedCfg.firstKey()
+                    && lastCfgRevision >= lastScaleUpRevision
+                    && lastCfgRevision >= lastScaleDownRevision
+            ) {
+                // It means that the zone was created but the data nodes value had not updated yet.
+                // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+                Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+                if (topologyEntry.value() == null) {

Review Comment:
   I agree that topologyEntry must be not null here. I added assert.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -600,24 +600,25 @@ private CompletableFuture<?> onTableCreate(ConfigurationNotificationEvent<TableV
             CatalogTableDescriptor tableDescriptor = toTableDescriptor(ctx.newValue());
             CatalogZoneDescriptor zoneDescriptor = getZoneDescriptor(tableDescriptor.zoneId());
 
-            List<Set<Assignment>> assignments;
+            CompletableFuture<List<Set<Assignment>>> assignments;

Review Comment:
   Renamed.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -947,7 +971,8 @@ private void initDataNodesAndTriggerKeysInMetaStorage(
                             revision
                     );
                 } else if (res.getAsBoolean()) {
-                    LOG.debug("Update zones' dataNodes value [zoneId = {}, dataNodes = {}, revision = {}]", zoneId, dataNodes, revision);
+                    LOG.debug("Update zones' dataNodes value [zoneId = {}, dataNodes = {}, revision = {}]",

Review Comment:
   I don't want to change it in my pr. So I have just reverted logging changes.



-- 
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 #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -649,19 +659,8 @@ public Set<String> dataNodes(int zoneId) {
         });
     }
 
-    /**
-     * Asynchronously gets data nodes of the zone using causality token.
-     *
-     * <p>The returned future can be completed with {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
-     * does not exist.
-     *
-     * @param causalityToken Causality token.
-     * @param zoneId Zone id.
-     * @return The future which will be completed with data nodes for the zoneId or with exception.
-     */
-    // TODO: Will be implemented in IGNITE-19506.
-    public CompletableFuture<Set<String>> dataNodes(long causalityToken, int zoneId) {
-        return null;
+    public Set<String> dataNodes(long causalityToken, int zoneId) {

Review Comment:
   Yes, I think I should get rid of this method if possible.



-- 
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] sergeyuttsel commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            Map<String, Map<String, String>> nodesAttributes
+    ) {
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.nodesAttributes = nodesAttributes;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the {@link DistributionZoneNotFoundException} if the zone with the provided {@code zoneId}
+     * does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The future which will be completed with data nodes for the zoneId or with exception.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        LOG.info("+++++++ dataNodes " + causalityToken + " " + zoneId);
+
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative number [zoneId=" + zoneId + '"');
+        }
+
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        // Get the latest configuration and configuration revision for a given causality token
+        Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = versionedCfg.floorEntry(causalityToken);
+
+        if (zoneLastCfgEntry == null) {
+            // It means that the zone does not exist on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+        ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+        String filter = zoneLastCfg.getFilter();
+
+        boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+        if (isZoneRemoved) {
+            // It means that the zone was removed on a given causality token.
+            throw new DistributionZoneNotFoundException(zoneId);
+        }
+
+        // Get revisions of the last scale up and scale down event which triggered immediate data nodes recalculation.
+        IgniteBiTuple<Long, Long> revisions = getRevisionsOfLastScaleUpAndScaleDownEvents(causalityToken, zoneId);
+        long lastScaleUpRevision = revisions.get1();
+        long lastScaleDownRevision = revisions.get2();
+
+        if (lastCfgRevision == versionedCfg.firstKey()
+                && lastCfgRevision >= lastScaleUpRevision
+                && lastCfgRevision >= lastScaleDownRevision
+        ) {
+            // It means that the zone was created but the data nodes value had not updated yet.
+            // So the data nodes value will be equals to the logical topology on the lastCfgRevision.
+
+            Entry topologyEntry = msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+            Set<NodeWithAttributes> logicalTopology = fromBytes(topologyEntry.value());
+
+            Set<Node> logicalTopologyNodes = logicalTopology.stream().map(n -> n.node()).collect(toSet());
+
+            Set<String> dataNodesNames = filterDataNodes(logicalTopologyNodes, filter, nodesAttributes);
+
+            return dataNodesNames;
+        }
+
+        LOG.info("+++++++ dataNodes lastScaleUpRevision " + lastScaleUpRevision);
+        LOG.info("+++++++ dataNodes lastScaleDownRevision " + lastScaleDownRevision);
+
+        ZoneState zoneState = zonesState.get(zoneId);
+
+        LOG.info("+++++++ dataNodes zoneState " + zoneState);
+
+        ConcurrentSkipListMap<Long, Augmentation> subAugmentationMap = null;
+
+        // On the data nodes recalculation we write new data nodes to the meta storage then clear the augmentation map.
+        // Therefore, first we need to read the augmentation map before it is cleared and then read the last data nodes value
+        // from the meta storage.
+        // The zoneState can be null if the zone was removed.
+        if (zoneState != null) {
+            subAugmentationMap = new ConcurrentSkipListMap<>(zoneState.topologyAugmentationMap()
+                    .headMap(causalityToken, true));
+        }
+
+        // Search the revisions of zoneScaleUpChangeTriggerKey and zoneScaleDownChangeTriggerKey with value greater or equals
+        // to expected one.
+        long scaleUpDataNodesRevision = searchTriggerKey(lastScaleUpRevision, zoneId, zoneScaleUpChangeTriggerKey(zoneId));
+        long scaleDownDataNodesRevision = searchTriggerKey(lastScaleDownRevision, zoneId, zoneScaleDownChangeTriggerKey(zoneId));
+
+        // Choose the highest revision.
+        long dataNodesRevision = max(causalityToken, max(scaleUpDataNodesRevision, scaleDownDataNodesRevision));
+
+        LOG.info("+++++++ dataNodes scaleUpDataNodesRevision " + scaleUpDataNodesRevision);
+        LOG.info("+++++++ dataNodes scaleDownDataNodesRevision " + scaleDownDataNodesRevision);
+
+        // Read data nodes value from the meta storage on dataNodesRevision and associated trigger keys.
+        Entry dataNodesEntry = msManager.getLocally(zoneDataNodesKey(zoneId), dataNodesRevision);
+        Entry scaleUpChangeTriggerKey = msManager.getLocally(zoneScaleUpChangeTriggerKey(zoneId), dataNodesRevision);
+        Entry scaleDownChangeTriggerKey = msManager.getLocally(zoneScaleDownChangeTriggerKey(zoneId), dataNodesRevision);
+
+        if (dataNodesEntry.value() == null) {
+            LOG.info("+++++++ dataNodes The zone was removed not idempotently");
+            // The zone was removed.
+            // In this case it is impossible to find out the data nodes value idempotently.
+            return emptySet();
+        }
+
+        Set<Node> baseDataNodes = DistributionZonesUtil.dataNodes(fromBytes(dataNodesEntry.value()));
+        long scaleUpTriggerRevision = bytesToLong(scaleUpChangeTriggerKey.value());
+        long scaleDownTriggerRevision = bytesToLong(scaleDownChangeTriggerKey.value());
+
+        LOG.info("+++++++ dataNodes scaleUpTriggerRevision " + scaleUpTriggerRevision);
+        LOG.info("+++++++ dataNodes scaleDownTriggerRevision " + scaleDownTriggerRevision);
+
+        LOG.info("+++++++ dataNodes baseDataNodes " + baseDataNodes);
+
+        Set<Node> finalDataNodes = new HashSet<>(baseDataNodes);
+
+        LOG.info("+++++++ dataNodes subAugmentationMap " + subAugmentationMap);
+
+        // If the subAugmentationMap is null then it means that the zone was removed. In this case all nodes from topologyAugmentationMap
+        // must be already written to the meta storage.
+        if (subAugmentationMap != null) {
+            // Update the data nodes set with pending data from augmentation map
+            subAugmentationMap.forEach((rev, augmentation) -> {
+                if (augmentation.addition() && rev > scaleUpTriggerRevision && rev <= lastScaleUpRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.add " + node);
+                        finalDataNodes.add(node);
+                    }
+                }
+
+                if (!augmentation.addition() && rev > scaleDownTriggerRevision && rev <= lastScaleDownRevision) {
+                    for (Node node : augmentation.nodes()) {
+                        LOG.info("+++++++ dataNodes finalDataNodes.remove " + node);
+                        finalDataNodes.remove(node);
+                    }
+                }
+            });
+        }
+
+        // Apply the filter to get the final data nodes set.
+        Set<String> result = filterDataNodes(finalDataNodes, filter, nodesAttributes);
+
+        LOG.info("+++++++ dataNodes result " + result);
+
+        return result;
+    }
+
+    /**
+     * These revisions correspond to the last configuration and topology events after which need to wait for the data nodes recalculation.
+     * These events are: a zone creation, changing a scale up timer to immediate, changing a scale down timer to immediate,
+     * changing a filter, deleting a zone, topology changes with the adding nodes, topology changes with removing nodes.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getRevisionsOfLastScaleUpAndScaleDownEvents(
+            long causalityToken,
+            int zoneId) {
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownConfigRevisions = getLastScaleUpAndScaleDownConfigRevisions(causalityToken, zoneId);
+
+        IgniteBiTuple<Long, Long> scaleUpAndScaleDownTopologyRevisions =
+                getLastScaleUpAndScaleDownTopologyRevisions(causalityToken, zoneId);
+
+        long lastScaleUpRevision = max(scaleUpAndScaleDownConfigRevisions.get1(), scaleUpAndScaleDownTopologyRevisions.get1());
+
+        long lastScaleDownRevision = max(scaleUpAndScaleDownConfigRevisions.get2(), scaleUpAndScaleDownTopologyRevisions.get2());
+
+        return new IgniteBiTuple<>(lastScaleUpRevision, lastScaleDownRevision);
+    }
+
+    /**
+     * Get revisions of the latest configuration change events which trigger immediate recalculation of the data nodes value.
+     *
+     * @param causalityToken causalityToken.
+     * @param zoneId zoneId.
+     * @return Revisions.
+     */
+    private IgniteBiTuple<Long, Long> getLastScaleUpAndScaleDownConfigRevisions(
+            long causalityToken,
+            int zoneId
+    ) {
+        ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = zonesVersionedCfg.get(zoneId);
+
+        Iterator<Map.Entry<Long, ZoneConfiguration>> reversedIterator = versionedCfg.headMap(causalityToken, true)
+                .descendingMap().entrySet().iterator();
+
+        Map.Entry<Long, ZoneConfiguration> entryNewerCfg = null;
+
+        long scaleUpRevision = 0;
+        long scaleDownRevision = 0;
+
+        // Iterate over zone configurations from newest to oldest.
+        while (reversedIterator.hasNext()) {
+            Map.Entry<Long, ZoneConfiguration> entryOlderCfg = reversedIterator.next();
+
+            ZoneConfiguration olderCfg = entryOlderCfg.getValue();
+
+            if (entryNewerCfg != null) {
+                boolean isScaleUpImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleUp() == IMMEDIATE_TIMER_VALUE;
+                boolean isScaleDownImmediate = entryNewerCfg.getValue().getDataNodesAutoAdjustScaleDown() == IMMEDIATE_TIMER_VALUE;
+
+                ZoneConfiguration newerCfg = entryNewerCfg.getValue();
+
+                if (scaleUpRevision == 0 && olderCfg.getDataNodesAutoAdjustScaleUp() != newerCfg.getDataNodesAutoAdjustScaleUp()

Review Comment:
   Yes, 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] alievmirza commented on a diff in pull request #2095: IGNITE-19506 Use data nodes from DistributionZoneManager with a causality token instead of BaselineManager#nodes

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.rebalance.DistributionZoneRebalanceEngine;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /**
+     * Map with states for distribution zones. States are needed to track nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * zoneId -> (revision -> zoneConfiguration).
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * Local mapping of {@code nodeId} -> node's attributes, where {@code nodeId} is a node id, that changes between restarts.
+     * This map is updated every time we receive a topology event in a {@code topologyWatchListener}.
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-19491 properly clean up this map
+     *
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/distribution-zones/tech-notes/filters.md">Filter documentation</a>
+     */
+    private Map<String, Map<String, String>> nodesAttributes;
+
+    /**
+     * The constructor.
+     *
+     * @param msManager msManager.
+     * @param vaultMgr vaultMgr.
+     * @param zonesState zonesState.
+     * @param nodesAttributes nodesAttributes.
+     */
+    public CausalityDataNodesEngine(
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,

Review Comment:
   I don't see any problems to call `new CausalityDataNodesEngine(this)`, we've already had the same logic for `DistributionZoneRebalanceEngine`



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