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

[GitHub] [ignite-3] sanpwc opened a new pull request, #2041: IGNITE-19440 Investigation.

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

   (no comment)


-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/build.gradle:
##########
@@ -33,6 +33,8 @@ dependencies {
     implementation project(':ignite-metastorage-api')
     implementation project(':ignite-vault')
     implementation project(':ignite-schema')
+    implementation project(':ignite-affinity')
+    implementation project(':ignite-baseline')

Review Comment:
   redundant import



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {

Review Comment:
   please add the javadoc



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :
+                                    ByteUtils.fromBytes(assignmentsBytes);
+
+                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
+                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+
+                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+
+                                int replicas = distributionZoneConfiguration.replicas().value();
+
+                                int partId = part;
+
+                                updatePendingAssignmentsKeys(
+                                        tableView.name(),
+                                        replicaGrpId,
+                                        dataNodes,
+                                        replicas,
+                                        evt.entryEvent().newEntry().revision(),
+                                        metaStorageManager,
+                                        part,
+                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
+                                ).exceptionally(e -> {
+                                    LOG.error(
+                                            "Exception on updating assignments for [table={}, partition={}]", e, tableView.name(),
+                                            partId
+                                    );
+
+                                    return null;
+                                });
+                            }
+                        }
+                    }
+
+                    return completedFuture(null);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                LOG.warn("Unable to process data nodes event", e);
+            }
+        };
+    }
+
+    /**
+     * Listener of replicas configuration changes.
+     *
+     * @param replicasCtx Replicas configuration event context.
+     * @return A future, which will be completed, when event processed by listener.
+     */
+    private CompletableFuture<?> onUpdateReplicas(ConfigurationNotificationEvent<Integer> replicasCtx) {
+        if (!busyLock.enterBusy()) {
+            return completedFuture(new NodeStoppingException());
+        }
+
+        try {
+            if (replicasCtx.oldValue() != null && replicasCtx.oldValue() > 0) {
+                DistributionZoneView zoneCfg = replicasCtx.newValue(DistributionZoneView.class);
+
+                List<TableConfiguration> tblsCfg = new ArrayList<>();
+
+                tablesConfiguration.tables().value().namedListKeys().forEach(tblName -> {
+                    if (tablesConfiguration.tables().get(tblName).zoneId().value().equals(zoneCfg.zoneId())) {
+                        tblsCfg.add(tablesConfiguration.tables().get(tblName));
+                    }
+                });
+
+                CompletableFuture<?>[] futs = new CompletableFuture[tblsCfg.size() * zoneCfg.partitions()];
+
+                int furCur = 0;
+
+                for (TableConfiguration tblCfg : tblsCfg) {
+
+                    LOG.info("Received update for replicas number [table={}, oldNumber={}, newNumber={}]",
+                            tblCfg.name().value(), replicasCtx.oldValue(), replicasCtx.newValue());
+
+                    int partCnt = zoneCfg.partitions();
+
+                    int newReplicas = replicasCtx.newValue();
+
+                    byte[] assignmentsBytes = ((ExtendedTableConfiguration) tblCfg).assignments().value();
+
+                    List<Set<Assignment>> tableAssignments = ByteUtils.fromBytes(assignmentsBytes);

Review Comment:
   according to the changes that you've made in `DistributionZoneRebalanceEngine#createDistributionZonesDataNodesListener`,
   `assignmentsBytes` could be null, that means that we will fail with NPE when pass null to `fromBytes`. Should we have the same check here? 
   
   Probably, Sergey asked the same question, but I didn't understand the answer, 
   
   what if `tableAssignments` is empty, what is expected behaviour for `tableAssignments.get(i)` in `updatePendingAssignmentsKeys` below?



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :
+                                    ByteUtils.fromBytes(assignmentsBytes);
+
+                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
+                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+
+                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+
+                                int replicas = distributionZoneConfiguration.replicas().value();
+
+                                int partId = part;
+
+                                updatePendingAssignmentsKeys(
+                                        tableView.name(),
+                                        replicaGrpId,
+                                        dataNodes,
+                                        replicas,
+                                        evt.entryEvent().newEntry().revision(),
+                                        metaStorageManager,
+                                        part,
+                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
+                                ).exceptionally(e -> {
+                                    LOG.error(
+                                            "Exception on updating assignments for [table={}, partition={}]", e, tableView.name(),
+                                            partId
+                                    );
+
+                                    return null;
+                                });
+                            }
+                        }
+                    }
+
+                    return completedFuture(null);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                LOG.warn("Unable to process data nodes event", e);
+            }
+        };
+    }
+
+    /**
+     * Listener of replicas configuration changes.
+     *
+     * @param replicasCtx Replicas configuration event context.
+     * @return A future, which will be completed, when event processed by listener.
+     */
+    private CompletableFuture<?> onUpdateReplicas(ConfigurationNotificationEvent<Integer> replicasCtx) {
+        if (!busyLock.enterBusy()) {
+            return completedFuture(new NodeStoppingException());
+        }
+
+        try {
+            if (replicasCtx.oldValue() != null && replicasCtx.oldValue() > 0) {
+                DistributionZoneView zoneCfg = replicasCtx.newValue(DistributionZoneView.class);
+
+                List<TableConfiguration> tblsCfg = new ArrayList<>();
+
+                tablesConfiguration.tables().value().namedListKeys().forEach(tblName -> {
+                    if (tablesConfiguration.tables().get(tblName).zoneId().value().equals(zoneCfg.zoneId())) {
+                        tblsCfg.add(tablesConfiguration.tables().get(tblName));
+                    }
+                });
+
+                CompletableFuture<?>[] futs = new CompletableFuture[tblsCfg.size() * zoneCfg.partitions()];
+
+                int furCur = 0;
+
+                for (TableConfiguration tblCfg : tblsCfg) {
+
+                    LOG.info("Received update for replicas number [table={}, oldNumber={}, newNumber={}]",
+                            tblCfg.name().value(), replicasCtx.oldValue(), replicasCtx.newValue());
+
+                    int partCnt = zoneCfg.partitions();
+
+                    int newReplicas = replicasCtx.newValue();
+
+                    byte[] assignmentsBytes = ((ExtendedTableConfiguration) tblCfg).assignments().value();
+
+                    List<Set<Assignment>> tableAssignments = ByteUtils.fromBytes(assignmentsBytes);
+
+                    for (int i = 0; i < partCnt; i++) {
+                        TablePartitionId replicaGrpId = new TablePartitionId(((ExtendedTableConfiguration) tblCfg).id().value(), i);
+
+                        futs[furCur++] = updatePendingAssignmentsKeys(
+                                tblCfg.name().value(),
+                                replicaGrpId,
+                                distributionZoneManager.getDataNodesByZoneId(zoneCfg.zoneId()),
+                                newReplicas,
+                                replicasCtx.storageRevision(), metaStorageManager, i, tableAssignments.get(i));

Review Comment:
   formatting 



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngineTest.java:
##########
@@ -487,40 +462,5 @@ private IgniteBiTuple<TableView, ExtendedTableConfiguration> mockTable(int table
         when(tableCfg.assignments()).thenReturn(assignmentValue);
 
         return new IgniteBiTuple<>(tableView, tableCfg);
-
-    }
-
-    private void mockTables(List<IgniteBiTuple<TableView, ExtendedTableConfiguration>> mockedTables) {

Review Comment:
   `DistributionZoneRebalanceEngineTest#mockTable` also can be removed



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :

Review Comment:
   I thought we have formatting like this for such code:
   ```
                               List<Set<Assignment>> tableAssignments = assignmentsBytes == null 
                                       ? Collections.emptyList() 
                                       : ByteUtils.fromBytes(assignmentsBytes);
   ```
   correct me please, if I'm wrong 



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -283,6 +288,15 @@ public DistributionZoneManager(
         );
 
         topVerTracker = new PendingComparableValuesTracker<>(0L);
+
+        rebalanceEngine = new DistributionZoneRebalanceEngine(
+                stopGuard,
+                busyLock,
+                zonesConfiguration,
+                tablesConfiguration,
+                metaStorageManager,
+                this

Review Comment:
   seems, that this is a bad practice to escape `this` in the constructor. Could we create `rebalanceEngine` in DZM.start? 



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :
+                                    ByteUtils.fromBytes(assignmentsBytes);
+
+                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
+                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+
+                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+
+                                int replicas = distributionZoneConfiguration.replicas().value();
+
+                                int partId = part;
+
+                                updatePendingAssignmentsKeys(
+                                        tableView.name(),
+                                        replicaGrpId,
+                                        dataNodes,
+                                        replicas,
+                                        evt.entryEvent().newEntry().revision(),
+                                        metaStorageManager,
+                                        part,
+                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
+                                ).exceptionally(e -> {
+                                    LOG.error(
+                                            "Exception on updating assignments for [table={}, partition={}]", e, tableView.name(),
+                                            partId
+                                    );
+
+                                    return null;
+                                });
+                            }
+                        }
+                    }
+
+                    return completedFuture(null);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                LOG.warn("Unable to process data nodes event", e);
+            }
+        };
+    }
+
+    /**
+     * Listener of replicas configuration changes.
+     *
+     * @param replicasCtx Replicas configuration event context.
+     * @return A future, which will be completed, when event processed by listener.
+     */
+    private CompletableFuture<?> onUpdateReplicas(ConfigurationNotificationEvent<Integer> replicasCtx) {
+        if (!busyLock.enterBusy()) {
+            return completedFuture(new NodeStoppingException());
+        }
+
+        try {
+            if (replicasCtx.oldValue() != null && replicasCtx.oldValue() > 0) {
+                DistributionZoneView zoneCfg = replicasCtx.newValue(DistributionZoneView.class);
+
+                List<TableConfiguration> tblsCfg = new ArrayList<>();
+
+                tablesConfiguration.tables().value().namedListKeys().forEach(tblName -> {
+                    if (tablesConfiguration.tables().get(tblName).zoneId().value().equals(zoneCfg.zoneId())) {
+                        tblsCfg.add(tablesConfiguration.tables().get(tblName));
+                    }
+                });
+
+                CompletableFuture<?>[] futs = new CompletableFuture[tblsCfg.size() * zoneCfg.partitions()];
+
+                int furCur = 0;
+
+                for (TableConfiguration tblCfg : tblsCfg) {
+
+                    LOG.info("Received update for replicas number [table={}, oldNumber={}, newNumber={}]",
+                            tblCfg.name().value(), replicasCtx.oldValue(), replicasCtx.newValue());
+
+                    int partCnt = zoneCfg.partitions();
+
+                    int newReplicas = replicasCtx.newValue();
+
+                    byte[] assignmentsBytes = ((ExtendedTableConfiguration) tblCfg).assignments().value();
+
+                    List<Set<Assignment>> tableAssignments = ByteUtils.fromBytes(assignmentsBytes);

Review Comment:
   at least, we need to have the same behaviour, but seems that we will fail with `IndexOutOfBoundsException`. Am I missing something? 



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {

Review Comment:
   Why? Don't you think that method name is self descriptive here?



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngineTest.java:
##########
@@ -487,40 +462,5 @@ private IgniteBiTuple<TableView, ExtendedTableConfiguration> mockTable(int table
         when(tableCfg.assignments()).thenReturn(assignmentValue);
 
         return new IgniteBiTuple<>(tableView, tableCfg);
-
-    }
-
-    private void mockTables(List<IgniteBiTuple<TableView, ExtendedTableConfiguration>> mockedTables) {

Review Comment:
   You are right. I've though I've 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] sanpwc merged pull request #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -283,6 +288,15 @@ public DistributionZoneManager(
         );
 
         topVerTracker = new PendingComparableValuesTracker<>(0L);
+
+        rebalanceEngine = new DistributionZoneRebalanceEngine(
+                stopGuard,
+                busyLock,
+                zonesConfiguration,
+                tablesConfiguration,
+                metaStorageManager,
+                this

Review Comment:
   Please, add comment then, that it is safe here to escape partially initialised object in such case and we do not violate some thread-safety rules like https://wiki.sei.cmu.edu/confluence/display/java/TSM01-J.+Do+not+let+the+this+reference+escape+during+object+construction, https://wiki.sei.cmu.edu/confluence/display/java/TSM03-J.+Do+not+publish+partially+initialized+objects



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :
+                                    ByteUtils.fromBytes(assignmentsBytes);
+
+                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
+                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+
+                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+
+                                int replicas = distributionZoneConfiguration.replicas().value();
+
+                                int partId = part;
+
+                                updatePendingAssignmentsKeys(
+                                        tableView.name(),
+                                        replicaGrpId,
+                                        dataNodes,
+                                        replicas,
+                                        evt.entryEvent().newEntry().revision(),
+                                        metaStorageManager,
+                                        part,
+                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
+                                ).exceptionally(e -> {
+                                    LOG.error(
+                                            "Exception on updating assignments for [table={}, partition={}]", e, tableView.name(),
+                                            partId
+                                    );
+
+                                    return null;
+                                });
+                            }
+                        }
+                    }
+
+                    return completedFuture(null);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                LOG.warn("Unable to process data nodes event", e);
+            }
+        };
+    }
+
+    /**
+     * Listener of replicas configuration changes.
+     *
+     * @param replicasCtx Replicas configuration event context.
+     * @return A future, which will be completed, when event processed by listener.
+     */
+    private CompletableFuture<?> onUpdateReplicas(ConfigurationNotificationEvent<Integer> replicasCtx) {
+        if (!busyLock.enterBusy()) {
+            return completedFuture(new NodeStoppingException());
+        }
+
+        try {
+            if (replicasCtx.oldValue() != null && replicasCtx.oldValue() > 0) {
+                DistributionZoneView zoneCfg = replicasCtx.newValue(DistributionZoneView.class);
+
+                List<TableConfiguration> tblsCfg = new ArrayList<>();
+
+                tablesConfiguration.tables().value().namedListKeys().forEach(tblName -> {
+                    if (tablesConfiguration.tables().get(tblName).zoneId().value().equals(zoneCfg.zoneId())) {
+                        tblsCfg.add(tablesConfiguration.tables().get(tblName));
+                    }
+                });
+
+                CompletableFuture<?>[] futs = new CompletableFuture[tblsCfg.size() * zoneCfg.partitions()];
+
+                int furCur = 0;
+
+                for (TableConfiguration tblCfg : tblsCfg) {
+
+                    LOG.info("Received update for replicas number [table={}, oldNumber={}, newNumber={}]",
+                            tblCfg.name().value(), replicasCtx.oldValue(), replicasCtx.newValue());
+
+                    int partCnt = zoneCfg.partitions();
+
+                    int newReplicas = replicasCtx.newValue();
+
+                    byte[] assignmentsBytes = ((ExtendedTableConfiguration) tblCfg).assignments().value();
+
+                    List<Set<Assignment>> tableAssignments = ByteUtils.fromBytes(assignmentsBytes);
+
+                    for (int i = 0; i < partCnt; i++) {
+                        TablePartitionId replicaGrpId = new TablePartitionId(((ExtendedTableConfiguration) tblCfg).id().value(), i);
+
+                        futs[furCur++] = updatePendingAssignmentsKeys(
+                                tblCfg.name().value(),
+                                replicaGrpId,
+                                distributionZoneManager.getDataNodesByZoneId(zoneCfg.zoneId()),
+                                newReplicas,
+                                replicasCtx.storageRevision(), metaStorageManager, i, tableAssignments.get(i));

Review Comment:
   Fixed.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :
+                                    ByteUtils.fromBytes(assignmentsBytes);
+
+                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
+                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+
+                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+
+                                int replicas = distributionZoneConfiguration.replicas().value();
+
+                                int partId = part;
+
+                                updatePendingAssignmentsKeys(
+                                        tableView.name(),
+                                        replicaGrpId,
+                                        dataNodes,
+                                        replicas,
+                                        evt.entryEvent().newEntry().revision(),
+                                        metaStorageManager,
+                                        part,
+                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
+                                ).exceptionally(e -> {
+                                    LOG.error(
+                                            "Exception on updating assignments for [table={}, partition={}]", e, tableView.name(),
+                                            partId
+                                    );
+
+                                    return null;
+                                });
+                            }
+                        }
+                    }
+
+                    return completedFuture(null);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                LOG.warn("Unable to process data nodes event", e);
+            }
+        };
+    }
+
+    /**
+     * Listener of replicas configuration changes.
+     *
+     * @param replicasCtx Replicas configuration event context.
+     * @return A future, which will be completed, when event processed by listener.
+     */
+    private CompletableFuture<?> onUpdateReplicas(ConfigurationNotificationEvent<Integer> replicasCtx) {
+        if (!busyLock.enterBusy()) {
+            return completedFuture(new NodeStoppingException());
+        }
+
+        try {
+            if (replicasCtx.oldValue() != null && replicasCtx.oldValue() > 0) {
+                DistributionZoneView zoneCfg = replicasCtx.newValue(DistributionZoneView.class);
+
+                List<TableConfiguration> tblsCfg = new ArrayList<>();
+
+                tablesConfiguration.tables().value().namedListKeys().forEach(tblName -> {
+                    if (tablesConfiguration.tables().get(tblName).zoneId().value().equals(zoneCfg.zoneId())) {
+                        tblsCfg.add(tablesConfiguration.tables().get(tblName));
+                    }
+                });
+
+                CompletableFuture<?>[] futs = new CompletableFuture[tblsCfg.size() * zoneCfg.partitions()];
+
+                int furCur = 0;
+
+                for (TableConfiguration tblCfg : tblsCfg) {
+
+                    LOG.info("Received update for replicas number [table={}, oldNumber={}, newNumber={}]",
+                            tblCfg.name().value(), replicasCtx.oldValue(), replicasCtx.newValue());
+
+                    int partCnt = zoneCfg.partitions();
+
+                    int newReplicas = replicasCtx.newValue();
+
+                    byte[] assignmentsBytes = ((ExtendedTableConfiguration) tblCfg).assignments().value();
+
+                    List<Set<Assignment>> tableAssignments = ByteUtils.fromBytes(assignmentsBytes);

Review Comment:
   Nope, it's never possible to have empty assignments, except test based scenarios where we use configuration injection. And it's not worth to fix it because @kgusakov will remove assignments from the configuration within his current ticket https://issues.apache.org/jira/browse/IGNITE-19405  



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/test/resources/META-INF/services/org.apache.ignite.configuration.ConfigurationModule:
##########
@@ -1 +0,0 @@
-org.apache.ignite.internal.storage.impl.TestPersistStorageConfigurationModule

Review Comment:
   What is this?Why did we remove this? 



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java:
##########
@@ -305,6 +305,7 @@ void testTwoQueuedRebalances() {
     void testThreeQueuedRebalances() throws Exception {
         await(createZone(nodes.get(0).distributionZoneManager, ZONE_1_NAME, 1, 1));
 
+        System.out.println("!!!");

Review Comment:
   ???



##########
modules/distribution-zones/src/test/resources/META-INF/services/org.apache.ignite.configuration.ConfigurationModule:
##########
@@ -1 +0,0 @@
-org.apache.ignite.internal.storage.impl.TestPersistStorageConfigurationModule

Review Comment:
   What is this? Why did we remove this? 



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/test/resources/META-INF/services/org.apache.ignite.configuration.ConfigurationModule:
##########
@@ -1 +0,0 @@
-org.apache.ignite.internal.storage.impl.TestPersistStorageConfigurationModule

Review Comment:
   Because of the test nature located in given module it's required to have both
   ```
   testImplementation (project(':ignite-storage-page-memory'))
   testImplementation(testFixtures(project(':ignite-storage-api')))
   ``` 
   see
   ```
           clusterCfgMgr = new ConfigurationManager(
                   List.of(DistributionZonesConfiguration.KEY),
                   Set.of(),
                   new TestConfigurationStorage(DISTRIBUTED),
                   List.of(),
                   List.of(PersistentPageMemoryDataStorageConfigurationSchema.class)
           );
   ```
   
   With the default provided in META-INF it's not possible to include `ignite-storage-...` hierarchy, following exception will be thrown:
   ```
   case already exists for value [aipersist]
   java.lang.IllegalStateException: case already exists for value [aipersist]
   	at com.facebook.presto.bytecode.BytecodeUtils.checkState(BytecodeUtils.java:89)
   	at org.apache.ignite.internal.configuration.asm.StringSwitchBuilder.addCase(StringSwitchBuilder.java:104)
   ```
   That's why given resource file was removed.



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java:
##########
@@ -305,6 +305,7 @@ void testTwoQueuedRebalances() {
     void testThreeQueuedRebalances() throws Exception {
         await(createZone(nodes.get(0).distributionZoneManager, ZONE_1_NAME, 1, 1));
 
+        System.out.println("!!!");

Review Comment:
   Debug code removed.



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java:
##########
@@ -305,6 +305,7 @@ void testTwoQueuedRebalances() {
     void testThreeQueuedRebalances() throws Exception {
         await(createZone(nodes.get(0).distributionZoneManager, ZONE_1_NAME, 1, 1));
 
+        System.out.println("!!!");

Review Comment:
   Debug code. Removed.



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :

Review Comment:
   There are no check-style errors, but I'll use your pattern.



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -283,6 +288,15 @@ public DistributionZoneManager(
         );
 
         topVerTracker = new PendingComparableValuesTracker<>(0L);
+
+        rebalanceEngine = new DistributionZoneRebalanceEngine(
+                stopGuard,
+                busyLock,
+                zonesConfiguration,
+                tablesConfiguration,
+                metaStorageManager,
+                this

Review Comment:
   Please, add comment then, that it is safe here to escape partially initialised object in such case and we do not violate some multithreading rules like https://wiki.sei.cmu.edu/confluence/display/java/TSM01-J.+Do+not+let+the+this+reference+escape+during+object+construction, https://wiki.sei.cmu.edu/confluence/display/java/TSM03-J.+Do+not+publish+partially+initialized+objects



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :
+                                    ByteUtils.fromBytes(assignmentsBytes);
+
+                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
+                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+
+                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+
+                                int replicas = distributionZoneConfiguration.replicas().value();
+
+                                int partId = part;
+
+                                updatePendingAssignmentsKeys(
+                                        tableView.name(),
+                                        replicaGrpId,
+                                        dataNodes,
+                                        replicas,
+                                        evt.entryEvent().newEntry().revision(),
+                                        metaStorageManager,
+                                        part,
+                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
+                                ).exceptionally(e -> {

Review Comment:
   Ok



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {

Review Comment:
   ok



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -283,6 +288,15 @@ public DistributionZoneManager(
         );
 
         topVerTracker = new PendingComparableValuesTracker<>(0L);
+
+        rebalanceEngine = new DistributionZoneRebalanceEngine(
+                stopGuard,
+                busyLock,
+                zonesConfiguration,
+                tablesConfiguration,
+                metaStorageManager,
+                this

Review Comment:
   Generally it's not a bad practice. It's about some multitheading threats that aren't applicable here. 



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -283,6 +288,15 @@ public DistributionZoneManager(
         );
 
         topVerTracker = new PendingComparableValuesTracker<>(0L);
+
+        rebalanceEngine = new DistributionZoneRebalanceEngine(
+                stopGuard,
+                busyLock,
+                zonesConfiguration,
+                tablesConfiguration,
+                metaStorageManager,
+                this

Review Comment:
   Yep, that should be possible. rebalanceEngine instantiation moved to start.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :
+                                    ByteUtils.fromBytes(assignmentsBytes);
+
+                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
+                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+
+                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+
+                                int replicas = distributionZoneConfiguration.replicas().value();
+
+                                int partId = part;
+
+                                updatePendingAssignmentsKeys(
+                                        tableView.name(),
+                                        replicaGrpId,
+                                        dataNodes,
+                                        replicas,
+                                        evt.entryEvent().newEntry().revision(),
+                                        metaStorageManager,
+                                        part,
+                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
+                                ).exceptionally(e -> {

Review Comment:
   Why is `tableAssignments.isEmpty()` done here and not done in onUpdateReplicas?



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -283,6 +288,15 @@ public DistributionZoneManager(
         );
 
         topVerTracker = new PendingComparableValuesTracker<>(0L);
+
+        rebalanceEngine = new DistributionZoneRebalanceEngine(
+                stopGuard,
+                busyLock,
+                zonesConfiguration,
+                tablesConfiguration,
+                metaStorageManager,
+                this

Review Comment:
   Comment added, warning suppressed.



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -283,6 +288,15 @@ public DistributionZoneManager(
         );
 
         topVerTracker = new PendingComparableValuesTracker<>(0L);
+
+        rebalanceEngine = new DistributionZoneRebalanceEngine(
+                stopGuard,
+                busyLock,
+                zonesConfiguration,
+                tablesConfiguration,
+                metaStorageManager,
+                this

Review Comment:
   Please, add comment then, that it is safe here to escape partially initialised object in such case and we do not violate some multithreading  rules https://wiki.sei.cmu.edu/confluence/display/java/TSM01-J.+Do+not+let+the+this+reference+escape+during+object+construction, https://wiki.sei.cmu.edu/confluence/display/java/TSM03-J.+Do+not+publish+partially+initialized+objects



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.rebalance;
+
+import static java.util.concurrent.CompletableFuture.allOf;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.dataNodes;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.extractZoneId;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.getZoneById;
+import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.updatePendingAssignmentsKeys;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableChange;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Zone rebalance manager.
+ */
+public class DistributionZoneRebalanceEngine {
+    /** The logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneRebalanceEngine.class);
+
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean stopGuard;
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Distribution zone configuration. */
+    private final DistributionZonesConfiguration zonesConfiguration;
+
+    /** Tables configuration. */
+    private final TablesConfiguration tablesConfiguration;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /** Meta storage listener for data nodes changes. */
+    private final WatchListener dataNodesListener;
+
+    /**
+     * The constructor.
+     *
+     * @param stopGuard Prevents double stopping of the component.
+     * @param busyLock Busy lock to stop synchronously.
+     * @param zonesConfiguration Distribution zone configuration.
+     * @param tablesConfiguration Tables configuration.
+     * @param metaStorageManager Meta Storage manager.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public DistributionZoneRebalanceEngine(
+            AtomicBoolean stopGuard,
+            IgniteSpinBusyLock busyLock,
+            DistributionZonesConfiguration zonesConfiguration,
+            TablesConfiguration tablesConfiguration,
+            MetaStorageManager metaStorageManager,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.stopGuard = stopGuard;
+        this.busyLock = busyLock;
+        this.zonesConfiguration = zonesConfiguration;
+        this.tablesConfiguration = tablesConfiguration;
+        this.metaStorageManager = metaStorageManager;
+        this.distributionZoneManager = distributionZoneManager;
+
+        this.dataNodesListener = createDistributionZonesDataNodesListener();
+    }
+
+    /**
+     * Starts the rebalance engine by registering corresponding meta storage and configuration listeners.
+     */
+    public void start() {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
+
+        try {
+            zonesConfiguration.distributionZones().any().replicas().listen(this::onUpdateReplicas);
+
+            // TODO: IGNITE-18694 - Recovery for the case when zones watch listener processed event but assignments were not updated.
+            metaStorageManager.registerPrefixWatch(zoneDataNodesKey(), dataNodesListener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Stops the rebalance engine by unregistering meta storage watches.
+     */
+    public void stop() {
+        if (!stopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();
+
+        metaStorageManager.unregisterWatch(dataNodesListener);
+    }
+
+    private WatchListener createDistributionZonesDataNodesListener() {
+        return new WatchListener() {
+            @Override
+            public CompletableFuture<Void> onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    return failedFuture(new NodeStoppingException());
+                }
+
+                try {
+                    byte[] dataNodesBytes = evt.entryEvent().newEntry().value();
+
+                    if (dataNodesBytes == null) {
+                        //The zone was removed so data nodes was removed too.
+                        return completedFuture(null);
+                    }
+
+                    NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
+
+                    int zoneId = extractZoneId(evt.entryEvent().newEntry().key());
+
+                    Set<String> dataNodes = dataNodes(ByteUtils.fromBytes(dataNodesBytes));
+
+                    for (int i = 0; i < tables.value().size(); i++) {
+                        TableView tableView = tables.value().get(i);
+
+                        int tableZoneId = tableView.zoneId();
+
+                        DistributionZoneConfiguration distributionZoneConfiguration =
+                                getZoneById(zonesConfiguration, tableZoneId);
+
+                        if (zoneId == tableZoneId) {
+                            TableConfiguration tableCfg = tables.get(tableView.name());
+
+                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+
+                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null ? Collections.emptyList() :
+                                    ByteUtils.fromBytes(assignmentsBytes);
+
+                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
+                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+
+                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+
+                                int replicas = distributionZoneConfiguration.replicas().value();
+
+                                int partId = part;
+
+                                updatePendingAssignmentsKeys(
+                                        tableView.name(),
+                                        replicaGrpId,
+                                        dataNodes,
+                                        replicas,
+                                        evt.entryEvent().newEntry().revision(),
+                                        metaStorageManager,
+                                        part,
+                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
+                                ).exceptionally(e -> {

Review Comment:
   Because it's harmless test related code that makes it possible to use table configuration injection in DistributionZoneRebalanceEngineTest
   ```
               @InjectConfiguration
                       ("mock.tables {"
                               + "table0 = { zoneId = 1 },"
                               + "table1 = { zoneId = 1 },"
                               + "table2 = { zoneId = 2 },"
                               + "table3 = { zoneId = 2 },"
                               + "table4 = { zoneId = 2 },"
                               + "table5 = { zoneId = 2 }}")
               TablesConfiguration tablesConfiguration
   ```
   Otherwise test will fail because of null assignments.



-- 
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 #2041: IGNITE-19440 Move onUpdateReplicas and distributionZonesDataNodesListener from table manager to distribution zones

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


##########
modules/distribution-zones/build.gradle:
##########
@@ -33,6 +33,8 @@ dependencies {
     implementation project(':ignite-metastorage-api')
     implementation project(':ignite-vault')
     implementation project(':ignite-schema')
+    implementation project(':ignite-affinity')
+    implementation project(':ignite-baseline')

Review Comment:
   Yep, removed.



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