denis-chudov commented on code in PR #2095:
URL: https://github.com/apache/ignite-3/pull/2095#discussion_r1274932579


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

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



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

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



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

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



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

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



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/causalitydatanodes/CausalityDataNodesEngine.java:
##########
@@ -0,0 +1,717 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.distributionzones.causalitydatanodes;
+
+import static java.lang.Math.max;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toSet;
+import static 
org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static 
org.apache.ignite.internal.distributionzones.DistributionZoneManager.IMMEDIATE_TIMER_VALUE;
+import static 
org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterDataNodes;
+import static 
org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneDataNodesKey;
+import static 
org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleDownChangeTriggerKey;
+import static 
org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneScaleUpChangeTriggerKey;
+import static 
org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zoneVersionedConfigurationKey;
+import static 
org.apache.ignite.internal.distributionzones.DistributionZonesUtil.zonesLogicalTopologyKey;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
+import static org.apache.ignite.internal.util.ByteUtils.fromBytes;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import 
org.apache.ignite.internal.distributionzones.DistributionZoneManager.Augmentation;
+import 
org.apache.ignite.internal.distributionzones.DistributionZoneManager.ZoneState;
+import org.apache.ignite.internal.distributionzones.DistributionZonesUtil;
+import org.apache.ignite.internal.distributionzones.Node;
+import org.apache.ignite.internal.distributionzones.NodeWithAttributes;
+import 
org.apache.ignite.internal.distributionzones.configuration.DistributionZoneView;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.DistributionZoneNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NodeStoppingException;
+
+/**
+ * Causality data nodes manager.
+ */
+public class CausalityDataNodesEngine {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock;
+
+    /** Meta Storage manager. */
+    private final MetaStorageManager msManager;
+
+    /** Vault manager. */
+    private final VaultManager vaultMgr;
+
+    /** Distribution zones manager. */
+    private final DistributionZoneManager distributionZoneManager;
+
+    /**
+     * Map with states for distribution zones. States are needed to track 
nodes that we want to add or remove from the data nodes,
+     * schedule and stop scale up and scale down processes.
+     */
+    private final Map<Integer, ZoneState> zonesState;
+
+    /**
+     * The map which contains configuration changes which trigger zone's data 
nodes recalculation.
+     * zoneId -> (revision -> zoneConfiguration).
+     * TODO IGNITE-20050 Clean up this map.
+     */
+    private final ConcurrentHashMap<Integer, ConcurrentSkipListMap<Long, 
ZoneConfiguration>> zonesVersionedCfg;
+
+    /**
+     * The constructor.
+     *
+     * @param busyLock Busy lock to stop synchronously.
+     * @param msManager Meta Storage manager.
+     * @param vaultMgr Vault manager.
+     * @param zonesState Map with states for distribution zones.
+     * @param distributionZoneManager Distribution zones manager.
+     */
+    public CausalityDataNodesEngine(
+            IgniteSpinBusyLock busyLock,
+            MetaStorageManager msManager,
+            VaultManager vaultMgr,
+            Map<Integer, ZoneState> zonesState,
+            DistributionZoneManager distributionZoneManager
+    ) {
+        this.busyLock = busyLock;
+        this.msManager = msManager;
+        this.vaultMgr = vaultMgr;
+        this.zonesState = zonesState;
+        this.distributionZoneManager = distributionZoneManager;
+
+        zonesVersionedCfg = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Gets data nodes of the zone using causality token.
+     *
+     * <p>Return data nodes or throw the exception:
+     * {@link IllegalArgumentException} if causalityToken or zoneId is not 
valid.
+     * {@link DistributionZoneNotFoundException} if the zone with the provided 
zoneId does not exist.
+     *
+     * @param causalityToken Causality token.
+     * @param zoneId Zone id.
+     * @return The data nodes for the zoneId.
+     */
+    public Set<String> dataNodes(long causalityToken, int zoneId) {
+        if (causalityToken < 1) {
+            throw new IllegalArgumentException("causalityToken must be greater 
then zero [causalityToken=" + causalityToken + '"');
+        }
+
+        if (zoneId < DEFAULT_ZONE_ID) {
+            throw new IllegalArgumentException("zoneId cannot be a negative 
number [zoneId=" + zoneId + '"');
+        }
+
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(NODE_STOPPING_ERR, new 
NodeStoppingException());
+        }
+
+        try {
+            ConcurrentSkipListMap<Long, ZoneConfiguration> versionedCfg = 
zonesVersionedCfg.get(zoneId);
+
+            // Get the latest configuration and configuration revision for a 
given causality token
+            Map.Entry<Long, ZoneConfiguration> zoneLastCfgEntry = 
versionedCfg.floorEntry(causalityToken);
+
+            if (zoneLastCfgEntry == null) {
+                // It means that the zone does not exist on a given causality 
token.
+                throw new DistributionZoneNotFoundException(zoneId);
+            }
+
+            long lastCfgRevision = zoneLastCfgEntry.getKey();
+
+            ZoneConfiguration zoneLastCfg = zoneLastCfgEntry.getValue();
+
+            String filter = zoneLastCfg.getFilter();
+
+            boolean isZoneRemoved = zoneLastCfg.getIsRemoved();
+
+            if (isZoneRemoved) {
+                // It means that the zone was removed on a given causality 
token.
+                throw new DistributionZoneNotFoundException(zoneId);
+            }
+
+            // Get revisions of the last scale up and scale down event which 
triggered immediate data nodes recalculation.
+            long lastScaleUpRevision = 
getRevisionsOfLastScaleUpEvent(causalityToken, zoneId);
+            long lastScaleDownRevision = 
getRevisionsOfLastScaleDownEvent(causalityToken, zoneId);
+
+            if (lastCfgRevision == versionedCfg.firstKey()
+                    && lastCfgRevision >= lastScaleUpRevision
+                    && lastCfgRevision >= lastScaleDownRevision
+            ) {
+                // It means that the zone was created but the data nodes value 
had not updated yet.
+                // So the data nodes value will be equals to the logical 
topology on the lastCfgRevision.
+
+                Entry topologyEntry = 
msManager.getLocally(zonesLogicalTopologyKey(), zoneLastCfgEntry.getKey());
+
+                if (topologyEntry.value() == null) {

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



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

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



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to