You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/08/16 16:55:11 UTC

[GitHub] [ignite-3] SammyVimes opened a new pull request, #1016: IGNITE-17196 In-memory partition rebalance

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

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


-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r955912407


##########
modules/core/src/main/java/org/apache/ignite/internal/util/ByteUtils.java:
##########
@@ -123,12 +123,12 @@ public static byte[] toBytes(Object obj) {
      * @param bytes Byte array.
      * @return Object.
      */
-    public static Object fromBytes(byte[] bytes) {
+    public static <T> T fromBytes(byte[] bytes) {
         try (
                 var bis = new ByteArrayInputStream(bytes);
                 var in = new ObjectInputStream(bis)
         ) {
-            return in.readObject();
+            return (T) in.readObject();

Review Comment:
   Well it's less code and the runtime implications are the same, are they not?



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957341460


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {

Review Comment:
   1) Changing to lambda.
   2) We only have listeners for groups ATM



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957411508


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/RebalanceRaftGroupEventsListener.java:
##########
@@ -252,92 +294,189 @@ private void scheduleChangePeers(List<PeerId> peers, long term) {
      */
     private void doOnNewPeersConfigurationApplied(List<PeerId> peers) {
         try {
-            Map<ByteArray, Entry> keys = metaStorageMgr.getAll(
+            ByteArray pendingPartAssignmentsKey = pendingPartAssignmentsKey(partId);
+            ByteArray stablePartAssignmentsKey = stablePartAssignmentsKey(partId);
+            ByteArray plannedPartAssignmentsKey = plannedPartAssignmentsKey(partId);
+            ByteArray switchReduceKey = switchReduceKey(partId);
+            ByteArray switchAppendKey = switchAppendKey(partId);
+
+            Map<ByteArray, Entry> values = metaStorageMgr.getAll(
                     Set.of(
-                            plannedPartAssignmentsKey(partId),
-                            pendingPartAssignmentsKey(partId),
-                            stablePartAssignmentsKey(partId))).get();
+                            plannedPartAssignmentsKey,
+                            pendingPartAssignmentsKey,
+                            stablePartAssignmentsKey,
+                            switchReduceKey,
+                            switchAppendKey
+                    )
+            ).get();
 
-            Entry plannedEntry = keys.get(plannedPartAssignmentsKey(partId));
+            Entry stableEntry = values.get(stablePartAssignmentsKey);
+            Entry pendingEntry = values.get(pendingPartAssignmentsKey);
+            Entry plannedEntry = values.get(plannedPartAssignmentsKey);
+            Entry switchReduceEntry = values.get(switchReduceKey);
+            Entry switchAppendEntry = values.get(switchAppendKey);
 
-            List<ClusterNode> appliedPeers = resolveClusterNodes(peers,
-                    keys.get(pendingPartAssignmentsKey(partId)).value(), keys.get(stablePartAssignmentsKey(partId)).value());
+            List<ClusterNode> calculatedAssignments = calculateAssignmentsFn.apply(tblConfiguration, partNum);
 
-            tblConfiguration.change(ch -> {
-                List<List<ClusterNode>> assignments =
-                        (List<List<ClusterNode>>) ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
-                assignments.set(partNum, appliedPeers);
-                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
-            }).get();
-
-            if (plannedEntry.value() != null) {
-                if (!metaStorageMgr.invoke(If.iif(
-                        revision(plannedPartAssignmentsKey(partId)).eq(plannedEntry.revision()),
-                        ops(
-                                put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                put(pendingPartAssignmentsKey(partId), plannedEntry.value()),
-                                remove(plannedPartAssignmentsKey(partId)))
-                                .yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            List<ClusterNode> stable = resolveClusterNodes(peers, pendingEntry.value(), stableEntry.value());
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            List<ClusterNode> retrievedSwitchReduce = readClusterNodes(switchReduceEntry);
+            List<ClusterNode> retrievedSwitchAppend = readClusterNodes(switchAppendEntry);
+            List<ClusterNode> retrievedStable = readClusterNodes(stableEntry);
 
-                LOG.info("Rebalance finished. Going to schedule next rebalance [partition={}, table={}, appliedPeers={}, plannedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers, ByteUtils.fromBytes(plannedEntry.value()));
-            } else {
-                if (!metaStorageMgr.invoke(If.iif(
-                        notExists(plannedPartAssignmentsKey(partId)),
-                        ops(put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                remove(pendingPartAssignmentsKey(partId))).yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            // Were reduced
+            List<ClusterNode> reducedNodes = subtract(retrievedSwitchReduce, stable);
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            // Were added
+            List<ClusterNode> addedNodes = subtract(stable, retrievedStable);
 
-                LOG.info("Rebalance finished [partition={}, table={}, appliedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers);
-            }
+            // For further reduction
+            List<ClusterNode> calculatedSwitchReduce = subtract(retrievedSwitchReduce, reducedNodes);
 
-            rebalanceAttempts.set(0);
-        } catch (InterruptedException | ExecutionException e) {
-            // TODO: IGNITE-14693
-            LOG.warn("Unable to commit partition configuration to metastore [table = {}, partition = {}]",
-                    e, tblConfiguration.name(), partNum);
-        }
-    }
+            // For further addition
+            List<ClusterNode> calculatedSwitchAppend = union(retrievedSwitchAppend, reducedNodes);
+            calculatedSwitchAppend = subtract(calculatedSwitchAppend, addedNodes);
+            calculatedSwitchAppend = intersect(calculatedAssignments, calculatedSwitchAppend);
 
-    private static List<ClusterNode> resolveClusterNodes(
-            List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
-        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+            var calculatedPendingReduction = subtract(stable, retrievedSwitchReduce);
 
-        if (pendingAssignments != null) {
-            ((List<ClusterNode>) ByteUtils.fromBytes(pendingAssignments)).forEach(n -> resolveRegistry.put(n.address(), n));
-        }
+            var calculatedPendingAddition = union(stable, reducedNodes);
+            calculatedPendingAddition = intersect(calculatedAssignments, calculatedPendingAddition);
 
-        if (stableAssignments != null) {
-            ((List<ClusterNode>) ByteUtils.fromBytes(stableAssignments)).forEach(n -> resolveRegistry.put(n.address(), n));
-        }
+            // eq(revision(assignments.stable), retrievedAssignmentsStable.revision)
+            SimpleCondition con1 = stableEntry.empty()
+                    ? notExists(stablePartAssignmentsKey) :
+                    revision(stablePartAssignmentsKey).eq(stableEntry.revision());
+
+            // eq(revision(assignments.pending), retrievedAssignmentsPending.revision)
+            SimpleCondition con2 = revision(pendingPartAssignmentsKey).eq(pendingEntry.revision());
 
-        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+            // eq(revision(assignments.switch.reduce), retrievedAssignmentsSwitchReduce.revision)
+            SimpleCondition con3 = switchReduceEntry.empty()
+                    ? notExists(switchReduceKey) : revision(switchReduceKey).eq(switchReduceEntry.revision());
 
-        for (PeerId p : peers) {
-            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+            // eq(revision(assignments.switch.append), retrievedAssignmentsSwitchAppend.revision)
+            SimpleCondition con4 = switchAppendEntry.empty()
+                    ? notExists(switchAppendKey) : revision(switchAppendKey).eq(switchAppendEntry.revision());
 
-            if (resolveRegistry.containsKey(addr)) {
-                resolvedNodes.add(resolveRegistry.get(addr));
+            Condition retryPreconditions = and(con1, and(con2, and(con3, con4)));
+
+            tblConfiguration.change(ch -> {
+                List<List<ClusterNode>> assignments = ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
+                assignments.set(partNum, stable);
+                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
+            }).get(10, TimeUnit.SECONDS);
+
+            Update successCase;
+            Update failCase;
+
+            byte[] stableByteArray = ByteUtils.toBytes(stable);
+            byte[] additionByteArray = ByteUtils.toBytes(calculatedPendingAddition);
+            byte[] reductionByteArray = ByteUtils.toBytes(calculatedPendingReduction);
+            byte[] switchReduceByteArray = ByteUtils.toBytes(calculatedSwitchReduce);
+            byte[] switchAppendByteArray = ByteUtils.toBytes(calculatedSwitchAppend);
+
+            if (!calculatedSwitchAppend.isEmpty()) {
+                successCase = ops(
+                        put(stablePartAssignmentsKey, stableByteArray),
+                        put(pendingPartAssignmentsKey, additionByteArray),
+                        put(switchReduceKey, switchReduceByteArray),
+                        put(switchAppendKey, switchAppendByteArray)
+                ).yield(SWITCH_APPEND_SUCCESS);
+                failCase = ops().yield(SWITCH_APPEND_FAIL);
+            } else if (!calculatedSwitchReduce.isEmpty()) {
+                successCase = ops(
+                        put(stablePartAssignmentsKey, stableByteArray),
+                        put(pendingPartAssignmentsKey, reductionByteArray),
+                        put(switchReduceKey, switchReduceByteArray),
+                        put(switchAppendKey, switchAppendByteArray)
+                ).yield(SWITCH_REDUCE_SUCCESS);
+                failCase = ops().yield(SWITCH_REDUCE_FAIL);
             } else {
-                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+                Condition con5;

Review Comment:
   well they're just parts of one big IF statement's condition basically, I don't think that they deserve a proper naming. Each of them have a comment that describes the condition



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957416780


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_REDUCE_PREFIX + partId);
+    }
+
+    /**
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchAppendKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_APPEND_PREFIX + partId);
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
+     *
+     * @param key Key.
+     * @return Table id.
+     */
+    public static UUID extractTableId(ByteArray key) {
+        return extractTableId(key, "");
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
      *
      * @param key Key.
+     * @param prefix Key prefix.
      * @return Table id.
      */
     public static UUID extractTableId(ByteArray key, String prefix) {
-        var strKey = key.toString();
+        String strKey = key.toString();

Review Comment:
   There is no method that accepts encoding



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957240736


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {
+        Integer partitions = tableCfg.partitions().value();
+        Integer replicas = tableCfg.replicas().value();
+        return AffinityUtils.calculateAssignments(baselineMgr.nodes(), partitions, replicas).get(partNum);

Review Comment:
   Oof, this smells so badly. If we only need assignments for a single partition, why do we calculate them for potentially hundreds or thousands irrelevant partitions?



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957252253


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -157,6 +171,12 @@ public class RebalanceUtil {
     /** Key prefix for stable assignments. */
     public static final String STABLE_ASSIGNMENTS_PREFIX = "assignments.stable.";
 
+    /** Key prefix for switch reduce assignments. */
+    public static final String ASSIGNMENTS_SWITCH_REDUCE_PREFIX = "assignments.switch.reduce.";

Review Comment:
   I guess it's not my business, but these names are too long. We will pollute metastorage with key names, it's horrible. I'm not blaming you, of course



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958116385


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_REDUCE_PREFIX + partId);
+    }
+
+    /**
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchAppendKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_APPEND_PREFIX + partId);
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
+     *
+     * @param key Key.
+     * @return Table id.
+     */
+    public static UUID extractTableId(ByteArray key) {
+        return extractTableId(key, "");
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
      *
      * @param key Key.
+     * @param prefix Key prefix.
      * @return Table id.
      */
     public static UUID extractTableId(ByteArray key, String prefix) {
-        var strKey = key.toString();
+        String strKey = key.toString();

Review Comment:
   My bad, sorry.



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957343512


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);

Review Comment:
   Network thread



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957362774


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -606,6 +708,32 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
         CompletableFuture.allOf(futures).join();
     }
 
+    /**
+     * Calculates the quantity of the data nodes for the partition of the table.
+     *
+     * @param tblId Table id.
+     * @param partId Partition id.
+     * @param partAssignments Partition assignments.
+     * @return A future that will hold the quantity of data nodes.
+     */
+    private CompletableFuture<Long> queryDataNodesCount(UUID tblId, int partId, List<ClusterNode> partAssignments) {

Review Comment:
   What can I say, Stream#count returns `long`...



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -606,6 +708,32 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
         CompletableFuture.allOf(futures).join();
     }
 
+    /**
+     * Calculates the quantity of the data nodes for the partition of the table.
+     *
+     * @param tblId Table id.
+     * @param partId Partition id.
+     * @param partAssignments Partition assignments.
+     * @return A future that will hold the quantity of data nodes.
+     */
+    private CompletableFuture<Long> queryDataNodesCount(UUID tblId, int partId, List<ClusterNode> partAssignments) {

Review Comment:
   What can I say, `Stream#count` returns `long`...



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957412068


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/RebalanceRaftGroupEventsListener.java:
##########
@@ -252,92 +294,189 @@ private void scheduleChangePeers(List<PeerId> peers, long term) {
      */
     private void doOnNewPeersConfigurationApplied(List<PeerId> peers) {
         try {
-            Map<ByteArray, Entry> keys = metaStorageMgr.getAll(
+            ByteArray pendingPartAssignmentsKey = pendingPartAssignmentsKey(partId);
+            ByteArray stablePartAssignmentsKey = stablePartAssignmentsKey(partId);
+            ByteArray plannedPartAssignmentsKey = plannedPartAssignmentsKey(partId);
+            ByteArray switchReduceKey = switchReduceKey(partId);
+            ByteArray switchAppendKey = switchAppendKey(partId);
+
+            Map<ByteArray, Entry> values = metaStorageMgr.getAll(
                     Set.of(
-                            plannedPartAssignmentsKey(partId),
-                            pendingPartAssignmentsKey(partId),
-                            stablePartAssignmentsKey(partId))).get();
+                            plannedPartAssignmentsKey,
+                            pendingPartAssignmentsKey,
+                            stablePartAssignmentsKey,
+                            switchReduceKey,
+                            switchAppendKey
+                    )
+            ).get();
 
-            Entry plannedEntry = keys.get(plannedPartAssignmentsKey(partId));
+            Entry stableEntry = values.get(stablePartAssignmentsKey);
+            Entry pendingEntry = values.get(pendingPartAssignmentsKey);
+            Entry plannedEntry = values.get(plannedPartAssignmentsKey);
+            Entry switchReduceEntry = values.get(switchReduceKey);
+            Entry switchAppendEntry = values.get(switchAppendKey);
 
-            List<ClusterNode> appliedPeers = resolveClusterNodes(peers,
-                    keys.get(pendingPartAssignmentsKey(partId)).value(), keys.get(stablePartAssignmentsKey(partId)).value());
+            List<ClusterNode> calculatedAssignments = calculateAssignmentsFn.apply(tblConfiguration, partNum);
 
-            tblConfiguration.change(ch -> {
-                List<List<ClusterNode>> assignments =
-                        (List<List<ClusterNode>>) ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
-                assignments.set(partNum, appliedPeers);
-                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
-            }).get();
-
-            if (plannedEntry.value() != null) {
-                if (!metaStorageMgr.invoke(If.iif(
-                        revision(plannedPartAssignmentsKey(partId)).eq(plannedEntry.revision()),
-                        ops(
-                                put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                put(pendingPartAssignmentsKey(partId), plannedEntry.value()),
-                                remove(plannedPartAssignmentsKey(partId)))
-                                .yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            List<ClusterNode> stable = resolveClusterNodes(peers, pendingEntry.value(), stableEntry.value());
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            List<ClusterNode> retrievedSwitchReduce = readClusterNodes(switchReduceEntry);
+            List<ClusterNode> retrievedSwitchAppend = readClusterNodes(switchAppendEntry);
+            List<ClusterNode> retrievedStable = readClusterNodes(stableEntry);
 
-                LOG.info("Rebalance finished. Going to schedule next rebalance [partition={}, table={}, appliedPeers={}, plannedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers, ByteUtils.fromBytes(plannedEntry.value()));
-            } else {
-                if (!metaStorageMgr.invoke(If.iif(
-                        notExists(plannedPartAssignmentsKey(partId)),
-                        ops(put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                remove(pendingPartAssignmentsKey(partId))).yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            // Were reduced
+            List<ClusterNode> reducedNodes = subtract(retrievedSwitchReduce, stable);
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            // Were added
+            List<ClusterNode> addedNodes = subtract(stable, retrievedStable);
 
-                LOG.info("Rebalance finished [partition={}, table={}, appliedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers);
-            }
+            // For further reduction
+            List<ClusterNode> calculatedSwitchReduce = subtract(retrievedSwitchReduce, reducedNodes);
 
-            rebalanceAttempts.set(0);
-        } catch (InterruptedException | ExecutionException e) {
-            // TODO: IGNITE-14693
-            LOG.warn("Unable to commit partition configuration to metastore [table = {}, partition = {}]",
-                    e, tblConfiguration.name(), partNum);
-        }
-    }
+            // For further addition
+            List<ClusterNode> calculatedSwitchAppend = union(retrievedSwitchAppend, reducedNodes);
+            calculatedSwitchAppend = subtract(calculatedSwitchAppend, addedNodes);
+            calculatedSwitchAppend = intersect(calculatedAssignments, calculatedSwitchAppend);
 
-    private static List<ClusterNode> resolveClusterNodes(
-            List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
-        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+            var calculatedPendingReduction = subtract(stable, retrievedSwitchReduce);
 
-        if (pendingAssignments != null) {
-            ((List<ClusterNode>) ByteUtils.fromBytes(pendingAssignments)).forEach(n -> resolveRegistry.put(n.address(), n));
-        }
+            var calculatedPendingAddition = union(stable, reducedNodes);
+            calculatedPendingAddition = intersect(calculatedAssignments, calculatedPendingAddition);
 
-        if (stableAssignments != null) {
-            ((List<ClusterNode>) ByteUtils.fromBytes(stableAssignments)).forEach(n -> resolveRegistry.put(n.address(), n));
-        }
+            // eq(revision(assignments.stable), retrievedAssignmentsStable.revision)
+            SimpleCondition con1 = stableEntry.empty()
+                    ? notExists(stablePartAssignmentsKey) :
+                    revision(stablePartAssignmentsKey).eq(stableEntry.revision());
+
+            // eq(revision(assignments.pending), retrievedAssignmentsPending.revision)
+            SimpleCondition con2 = revision(pendingPartAssignmentsKey).eq(pendingEntry.revision());
 
-        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+            // eq(revision(assignments.switch.reduce), retrievedAssignmentsSwitchReduce.revision)
+            SimpleCondition con3 = switchReduceEntry.empty()
+                    ? notExists(switchReduceKey) : revision(switchReduceKey).eq(switchReduceEntry.revision());
 
-        for (PeerId p : peers) {
-            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+            // eq(revision(assignments.switch.append), retrievedAssignmentsSwitchAppend.revision)
+            SimpleCondition con4 = switchAppendEntry.empty()
+                    ? notExists(switchAppendKey) : revision(switchAppendKey).eq(switchAppendEntry.revision());
 
-            if (resolveRegistry.containsKey(addr)) {
-                resolvedNodes.add(resolveRegistry.get(addr));
+            Condition retryPreconditions = and(con1, and(con2, and(con3, con4)));
+
+            tblConfiguration.change(ch -> {
+                List<List<ClusterNode>> assignments = ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
+                assignments.set(partNum, stable);
+                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
+            }).get(10, TimeUnit.SECONDS);
+
+            Update successCase;
+            Update failCase;
+
+            byte[] stableByteArray = ByteUtils.toBytes(stable);
+            byte[] additionByteArray = ByteUtils.toBytes(calculatedPendingAddition);
+            byte[] reductionByteArray = ByteUtils.toBytes(calculatedPendingReduction);
+            byte[] switchReduceByteArray = ByteUtils.toBytes(calculatedSwitchReduce);
+            byte[] switchAppendByteArray = ByteUtils.toBytes(calculatedSwitchAppend);
+
+            if (!calculatedSwitchAppend.isEmpty()) {
+                successCase = ops(
+                        put(stablePartAssignmentsKey, stableByteArray),
+                        put(pendingPartAssignmentsKey, additionByteArray),
+                        put(switchReduceKey, switchReduceByteArray),
+                        put(switchAppendKey, switchAppendByteArray)
+                ).yield(SWITCH_APPEND_SUCCESS);
+                failCase = ops().yield(SWITCH_APPEND_FAIL);
+            } else if (!calculatedSwitchReduce.isEmpty()) {
+                successCase = ops(
+                        put(stablePartAssignmentsKey, stableByteArray),
+                        put(pendingPartAssignmentsKey, reductionByteArray),
+                        put(switchReduceKey, switchReduceByteArray),
+                        put(switchAppendKey, switchAppendByteArray)
+                ).yield(SWITCH_REDUCE_SUCCESS);
+                failCase = ops().yield(SWITCH_REDUCE_FAIL);
             } else {
-                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+                Condition con5;
+                if (plannedEntry.value() != null) {
+                    // eq(revision(partition.assignments.planned), plannedEntry.revision)
+                    con5 = revision(plannedPartAssignmentsKey).eq(plannedEntry.revision());
+
+                    successCase = ops(
+                            put(stablePartAssignmentsKey, ByteUtils.toBytes(stable)),
+                            put(pendingPartAssignmentsKey, plannedEntry.value()),
+                            remove(plannedPartAssignmentsKey)
+                    ).yield(SCHEDULE_PENDING_REBALANCE_SUCCESS);
+
+                    failCase = ops().yield(SCHEDULE_PENDING_REBALANCE_FAIL);
+                } else {
+                    // notExists(partition.assignments.planned)
+                    con5 = notExists(plannedPartAssignmentsKey);
+
+                    successCase = ops(
+                            put(stablePartAssignmentsKey, ByteUtils.toBytes(stable)),
+                            remove(pendingPartAssignmentsKey)
+                    ).yield(FINISH_REBALANCE_SUCCESS);
+
+                    failCase = ops().yield(FINISH_REBALANCE_FAIL);
+                }
+
+                retryPreconditions = and(retryPreconditions, con5);
             }
-        }
 
-        return resolvedNodes;
+            int res = metaStorageMgr.invoke(If.iif(retryPreconditions, successCase, failCase)).get().getAsInt();

Review Comment:
   Added a 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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957377854


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {
+        Integer partitions = tableCfg.partitions().value();
+        Integer replicas = tableCfg.replicas().value();
+        return AffinityUtils.calculateAssignments(baselineMgr.nodes(), partitions, replicas).get(partNum);

Review Comment:
   Right, I'll rewrite 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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957415506


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_REDUCE_PREFIX + partId);
+    }
+
+    /**
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchAppendKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_APPEND_PREFIX + partId);
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
+     *
+     * @param key Key.
+     * @return Table id.
+     */
+    public static UUID extractTableId(ByteArray key) {
+        return extractTableId(key, "");
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
      *
      * @param key Key.
+     * @param prefix Key prefix.
      * @return Table id.
      */
     public static UUID extractTableId(ByteArray key, String prefix) {
-        var strKey = key.toString();
+        String strKey = key.toString();
 
         return UUID.fromString(strKey.substring(prefix.length(), strKey.indexOf("_part_")));

Review Comment:
   They do have fixed length, but what do you mean?



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958225959


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);
+
+                    if (table != null) {
+                        MvTableStorage storage = table.internalTable().storage();
+
+                        MvPartitionStorage mvPartition = storage.getMvPartition(partitionId);
+
+                        if (mvPartition != null) {
+                            // If applied index of a storage is greater than 0,
+                            // then there is data
+                            contains = mvPartition.lastAppliedIndex() > 0;

Review Comment:
   Well if it is 0 at this moment, then it should be false. We shouldn't wait for recovery, it should be something else, I think



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

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

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


[GitHub] [ignite-3] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958231725


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/SimpleCondition.java:
##########
@@ -77,6 +77,8 @@ public long revision() {
          * @throws IllegalStateException In the case when the condition is already defined.
          */
         public SimpleCondition eq(long rev) {
+            assert rev > 0 : "Revision must be positive.";

Review Comment:
   Weeeell, there are no methods for value comparison in `Requires`...



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958109473


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
+
+                                        if (majorityAvailable) {
+                                            String partitionId = partitionRaftGroupName(tblId, partId);
+                                            RebalanceUtil.startPeerRemoval(partitionId, localMember, metaStorageMgr);
+                                            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.";
+                                            throw new IgniteInternalException(msg);
+                                        }
+                                    });
+                                } else {
+                                    fut = CompletableFuture.completedFuture(true);
                                 }
+
+                                return fut.thenComposeAsync(startGroup -> {

Review Comment:
   Can we use "composeAsync" or 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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957239916


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {
+        Integer partitions = tableCfg.partitions().value();

Review Comment:
   Quick note, it's better when you use "tableCfg.value()" and then access its properties. It's faster and it doesn't involve boxing.



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958141742


##########
modules/affinity/src/main/java/org/apache/ignite/internal/affinity/AffinityUtils.java:
##########
@@ -17,15 +17,43 @@
 
 package org.apache.ignite.internal.affinity;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
 import org.apache.ignite.network.ClusterNode;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * Stateless affinity utils that produces helper methods for an affinity assignments calculation.
  */
 public class AffinityUtils {
+    /**
+     * Calculates affinity assignments.
+     *
+     * @param partitions Partitions count.
+     * @param replicas   Replicas count.

Review Comment:
   ```suggestion
        * @param replicas Replicas count.
   ```



##########
modules/affinity/src/main/java/org/apache/ignite/internal/affinity/AffinityUtils.java:
##########
@@ -17,15 +17,43 @@
 
 package org.apache.ignite.internal.affinity;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
 import org.apache.ignite.network.ClusterNode;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * Stateless affinity utils that produces helper methods for an affinity assignments calculation.
  */
 public class AffinityUtils {
+    /**
+     * Calculates affinity assignments.
+     *
+     * @param partitions Partitions count.
+     * @param replicas   Replicas count.
+     * @param aggregator Function that creates a collection for the partition assignments.
+     * @return List nodes by partition.
+     */
+    public static <T extends Collection<ClusterNode>> List<T> calculateAssignments(
+            @NotNull Collection<ClusterNode> baselineNodes,
+            int partitions,
+            int replicas,
+            Function<Integer, T> aggregator

Review Comment:
   There's an `IntFunction` in Java



-- 
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] sashapolo commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r955906440


##########
modules/core/src/main/java/org/apache/ignite/internal/util/ByteUtils.java:
##########
@@ -123,12 +123,12 @@ public static byte[] toBytes(Object obj) {
      * @param bytes Byte array.
      * @return Object.
      */
-    public static Object fromBytes(byte[] bytes) {
+    public static <T> T fromBytes(byte[] bytes) {
         try (
                 var bis = new ByteArrayInputStream(bytes);
                 var in = new ObjectInputStream(bis)
         ) {
-            return in.readObject();
+            return (T) in.readObject();

Review Comment:
   This is an unsafe cast, I would prefer to leave the current approach with explicit casting



##########
modules/network/src/main/java/org/apache/ignite/internal/network/netty/NettyUtils.java:
##########
@@ -38,18 +41,21 @@ public class NettyUtils {
      * @return CompletableFuture.
      */
     public static <T, R, F extends Future<R>> CompletableFuture<T> toCompletableFuture(
-            F nettyFuture,
+            @Schedule F nettyFuture,
             Function<F, T> mapper
     ) {
         var fut = new CompletableFuture<T>();
 
-        nettyFuture.addListener((F future) -> {
-            if (future.isSuccess()) {
-                fut.complete(mapper.apply(future));
-            } else if (future.isCancelled()) {
-                fut.cancel(true);
-            } else {
-                fut.completeExceptionally(future.cause());
+        nettyFuture.addListener(new GenericFutureListener<F>() {
+            @Override
+            public void operationComplete(@Execute F future) throws Exception {

Review Comment:
   Is it impossible to add annotations to lambda parameters?



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958426605


##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java:
##########
@@ -603,7 +604,17 @@ else if (resp0.errorCode() == RaftError.EBUSY.getNumber() ||
                         resp0.errorCode() == (RaftError.EAGAIN.getNumber()) ||
                         resp0.errorCode() == (RaftError.ENOENT.getNumber())) { // Possibly a node has not been started.
                         executor.schedule(() -> {
-                            sendWithRetry(peer, req, stopTime, fut);
+                            Peer targetPeer = peer;
+
+                            if (resp0.errorCode() == RaftError.ENOENT.getNumber()) {

Review Comment:
   I don't see how I can test this separately so it will be just a little commit with 3 lines of code and this will also delay the merging of this ticket, so...



-- 
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] sk0x50 commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
sk0x50 commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958413693


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java:
##########
@@ -533,6 +538,7 @@ private final class Watcher extends Thread {
              * @param lsnr   The listener which receives and handles watch updates.
              */
             Watcher(Cursor<WatchEvent> cursor, WatchListener lsnr) {
+                setName("ms-watcher-" + localNodeName);

Review Comment:
   It would be better to use `IgniteThread` instead of `Watcher extends Thread` I think. Will file a ticket for 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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958104032


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);

Review Comment:
   Is this a operation fast enough to not stall the network thread?



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958123515


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()

Review Comment:
   Oh, it's an `else` branch! That is SO hard to read.
   You know what? We can do what Mockito does, in a way. Add methods like `then` and `els` just for sugar.



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958116967


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {

Review Comment:
   You should comment this comment so that it won't feel like a real commented code



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958102248


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/SimpleCondition.java:
##########
@@ -77,6 +77,8 @@ public long revision() {
          * @throws IllegalStateException In the case when the condition is already defined.
          */
         public SimpleCondition eq(long rev) {
+            assert rev > 0 : "Revision must be positive.";

Review Comment:
   2. Yes, I mean something like that. Is this even possible? I guess asserts are 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] sk0x50 commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
sk0x50 commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958400951


##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java:
##########
@@ -603,7 +604,17 @@ else if (resp0.errorCode() == RaftError.EBUSY.getNumber() ||
                         resp0.errorCode() == (RaftError.EAGAIN.getNumber()) ||
                         resp0.errorCode() == (RaftError.ENOENT.getNumber())) { // Possibly a node has not been started.
                         executor.schedule(() -> {
-                            sendWithRetry(peer, req, stopTime, fut);
+                            Peer targetPeer = peer;
+
+                            if (resp0.errorCode() == RaftError.ENOENT.getNumber()) {

Review Comment:
   Does this change belong to the ticket itself? It seems to me, this is a separate issue that should be addressed by a separate ticket.



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958102883


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java:
##########
@@ -0,0 +1,354 @@
+/*
+ * 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.runner.app;
+
+import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
+import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * These tests check in-memory node restart scenarios.
+ */
+public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
+    /** Default node port. */
+    private static final int DEFAULT_NODE_PORT = 3344;
+
+    /** Value producer for table data, is used to create data and check it later. */
+    private static final IntFunction<String> VALUE_PRODUCER = i -> "val " + i;
+
+    /** Prefix for full table name. */
+    private static final String SCHEMA_PREFIX = "PUBLIC.";
+
+    /** Test table name. */
+    private static final String TABLE_NAME = "Table1";
+
+    /** Nodes bootstrap configuration pattern. */
+    private static final String NODE_BOOTSTRAP_CFG = "{\n"
+            + "  network.port: {},\n"
+            + "  network.nodeFinder.netClusterNodes: {}\n"
+            + "}";
+
+    /** Cluster nodes. */
+    private static final List<Ignite> CLUSTER_NODES = new ArrayList<>();
+
+    private static final List<String> CLUSTER_NODES_NAMES = new ArrayList<>();
+
+    /**
+     * Stops all started nodes.
+     */
+    @AfterEach
+    public void afterEach() throws Exception {
+        var closeables = new ArrayList<AutoCloseable>();
+
+        for (String name : CLUSTER_NODES_NAMES) {
+            if (name != null) {
+                closeables.add(() -> IgnitionManager.stop(name));
+            }
+        }
+
+        CLUSTER_NODES.clear();
+        CLUSTER_NODES_NAMES.clear();
+
+        IgniteUtils.closeAll(closeables);
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @param nodeName Node name.
+     * @param cfgString Configuration string.
+     * @param workDir Working directory.
+     * @return Created node instance.
+     */
+    private static IgniteImpl startNode(int idx, String nodeName, @Nullable String cfgString, Path workDir) {
+        assertTrue(CLUSTER_NODES.size() == idx || CLUSTER_NODES.get(idx) == null);
+
+        CLUSTER_NODES_NAMES.add(idx, nodeName);
+
+        CompletableFuture<Ignite> future = IgnitionManager.start(nodeName, cfgString, workDir.resolve(nodeName));
+
+        if (CLUSTER_NODES.isEmpty()) {
+            IgnitionManager.init(nodeName, List.of(nodeName), "cluster");
+        }
+
+        assertThat(future, willCompleteSuccessfully());
+
+        Ignite ignite = future.join();
+
+        CLUSTER_NODES.add(idx, ignite);
+
+        return (IgniteImpl) ignite;
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param testInfo Test info.
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @return Created node instance.
+     */
+    private IgniteImpl startNode(TestInfo testInfo, int idx) {
+        int port = DEFAULT_NODE_PORT + idx;
+        String nodeName = testNodeName(testInfo, port);
+        String cfgString = configurationString(idx, null, null);
+
+        return startNode(idx, nodeName, cfgString, workDir.resolve(nodeName));
+    }
+
+    /**
+     * Build a configuration string.
+     *
+     * @param idx Node index.
+     * @param cfg Optional configuration string.
+     * @param predefinedPort Predefined port.
+     * @return Configuration string.
+     */
+    private static String configurationString(int idx, @Nullable String cfg, @Nullable Integer predefinedPort) {
+        int port = predefinedPort == null ? DEFAULT_NODE_PORT + idx : predefinedPort;
+        int connectPort = predefinedPort == null ? DEFAULT_NODE_PORT : predefinedPort;
+        String connectAddr = "[\"localhost:" + connectPort + "\"]";

Review Comment:
   Now I see, thank you!



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957365329


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {
+        Integer partitions = tableCfg.partitions().value();

Review Comment:
   Thanks!



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

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

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


[GitHub] [ignite-3] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957408631


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/RebalanceRaftGroupEventsListener.java:
##########
@@ -252,92 +294,189 @@ private void scheduleChangePeers(List<PeerId> peers, long term) {
      */
     private void doOnNewPeersConfigurationApplied(List<PeerId> peers) {
         try {
-            Map<ByteArray, Entry> keys = metaStorageMgr.getAll(
+            ByteArray pendingPartAssignmentsKey = pendingPartAssignmentsKey(partId);
+            ByteArray stablePartAssignmentsKey = stablePartAssignmentsKey(partId);
+            ByteArray plannedPartAssignmentsKey = plannedPartAssignmentsKey(partId);
+            ByteArray switchReduceKey = switchReduceKey(partId);
+            ByteArray switchAppendKey = switchAppendKey(partId);
+
+            Map<ByteArray, Entry> values = metaStorageMgr.getAll(
                     Set.of(
-                            plannedPartAssignmentsKey(partId),
-                            pendingPartAssignmentsKey(partId),
-                            stablePartAssignmentsKey(partId))).get();
+                            plannedPartAssignmentsKey,
+                            pendingPartAssignmentsKey,
+                            stablePartAssignmentsKey,
+                            switchReduceKey,
+                            switchAppendKey
+                    )
+            ).get();
 
-            Entry plannedEntry = keys.get(plannedPartAssignmentsKey(partId));
+            Entry stableEntry = values.get(stablePartAssignmentsKey);
+            Entry pendingEntry = values.get(pendingPartAssignmentsKey);
+            Entry plannedEntry = values.get(plannedPartAssignmentsKey);
+            Entry switchReduceEntry = values.get(switchReduceKey);
+            Entry switchAppendEntry = values.get(switchAppendKey);
 
-            List<ClusterNode> appliedPeers = resolveClusterNodes(peers,
-                    keys.get(pendingPartAssignmentsKey(partId)).value(), keys.get(stablePartAssignmentsKey(partId)).value());
+            List<ClusterNode> calculatedAssignments = calculateAssignmentsFn.apply(tblConfiguration, partNum);
 
-            tblConfiguration.change(ch -> {
-                List<List<ClusterNode>> assignments =
-                        (List<List<ClusterNode>>) ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
-                assignments.set(partNum, appliedPeers);
-                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
-            }).get();
-
-            if (plannedEntry.value() != null) {
-                if (!metaStorageMgr.invoke(If.iif(
-                        revision(plannedPartAssignmentsKey(partId)).eq(plannedEntry.revision()),
-                        ops(
-                                put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                put(pendingPartAssignmentsKey(partId), plannedEntry.value()),
-                                remove(plannedPartAssignmentsKey(partId)))
-                                .yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            List<ClusterNode> stable = resolveClusterNodes(peers, pendingEntry.value(), stableEntry.value());
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            List<ClusterNode> retrievedSwitchReduce = readClusterNodes(switchReduceEntry);
+            List<ClusterNode> retrievedSwitchAppend = readClusterNodes(switchAppendEntry);
+            List<ClusterNode> retrievedStable = readClusterNodes(stableEntry);
 
-                LOG.info("Rebalance finished. Going to schedule next rebalance [partition={}, table={}, appliedPeers={}, plannedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers, ByteUtils.fromBytes(plannedEntry.value()));
-            } else {
-                if (!metaStorageMgr.invoke(If.iif(
-                        notExists(plannedPartAssignmentsKey(partId)),
-                        ops(put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                remove(pendingPartAssignmentsKey(partId))).yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            // Were reduced
+            List<ClusterNode> reducedNodes = subtract(retrievedSwitchReduce, stable);
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            // Were added
+            List<ClusterNode> addedNodes = subtract(stable, retrievedStable);
 
-                LOG.info("Rebalance finished [partition={}, table={}, appliedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers);
-            }
+            // For further reduction
+            List<ClusterNode> calculatedSwitchReduce = subtract(retrievedSwitchReduce, reducedNodes);
 
-            rebalanceAttempts.set(0);
-        } catch (InterruptedException | ExecutionException e) {
-            // TODO: IGNITE-14693
-            LOG.warn("Unable to commit partition configuration to metastore [table = {}, partition = {}]",
-                    e, tblConfiguration.name(), partNum);
-        }
-    }
+            // For further addition
+            List<ClusterNode> calculatedSwitchAppend = union(retrievedSwitchAppend, reducedNodes);
+            calculatedSwitchAppend = subtract(calculatedSwitchAppend, addedNodes);
+            calculatedSwitchAppend = intersect(calculatedAssignments, calculatedSwitchAppend);
 
-    private static List<ClusterNode> resolveClusterNodes(
-            List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
-        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+            var calculatedPendingReduction = subtract(stable, retrievedSwitchReduce);

Review Comment:
   Right, forgot to clean them up



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957417282


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.

Review Comment:
   Right



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958437208


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java:
##########
@@ -533,6 +538,7 @@ private final class Watcher extends Thread {
              * @param lsnr   The listener which receives and handles watch updates.
              */
             Watcher(Cursor<WatchEvent> cursor, WatchListener lsnr) {
+                setName("ms-watcher-" + localNodeName);

Review Comment:
   I agree, we don't need to extend java's basic threads when we have ignite thread



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r956003401


##########
modules/core/src/main/java/org/apache/ignite/internal/util/ByteUtils.java:
##########
@@ -123,12 +123,12 @@ public static byte[] toBytes(Object obj) {
      * @param bytes Byte array.
      * @return Object.
      */
-    public static Object fromBytes(byte[] bytes) {
+    public static <T> T fromBytes(byte[] bytes) {
         try (
                 var bis = new ByteArrayInputStream(bytes);
                 var in = new ObjectInputStream(bis)
         ) {
-            return in.readObject();
+            return (T) in.readObject();

Review Comment:
   I'm against such code, but for other reasons.
   Technically, both approaches work exactly the same here. This method will return Object without cast, because no type information is available to it. It's all just sugar.



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958103593


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {

Review Comment:
   > We only have listeners for groups ATM
   
   Did not expect that...



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958118884


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(

Review Comment:
   Calling it `if` is like writing `int number;` and `String string`, it doesn't explain the purpose of the value. It may be anything like `resultingOperation` or I don't know



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958406548


##########
modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeMessageTypes.java:
##########
@@ -24,7 +24,7 @@
 /**
  * Message types for the Compute module.
  */
-@MessageGroup(groupName = "ComputeMessages", groupType = 6)
+@MessageGroup(groupType = 6, groupName = "ComputeMessages")

Review Comment:
   groupType is first everywhere else, just a little style change



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r956003401


##########
modules/core/src/main/java/org/apache/ignite/internal/util/ByteUtils.java:
##########
@@ -123,12 +123,12 @@ public static byte[] toBytes(Object obj) {
      * @param bytes Byte array.
      * @return Object.
      */
-    public static Object fromBytes(byte[] bytes) {
+    public static <T> T fromBytes(byte[] bytes) {
         try (
                 var bis = new ByteArrayInputStream(bytes);
                 var in = new ObjectInputStream(bis)
         ) {
-            return in.readObject();
+            return (T) in.readObject();

Review Comment:
   I'm against such code, but technically, both approaches are absolutely the same here. This method will return Object without cast, because no type information is available to it. It's all just sugar.



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957342569


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java:
##########
@@ -0,0 +1,354 @@
+/*
+ * 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.runner.app;
+
+import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
+import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * These tests check in-memory node restart scenarios.
+ */
+public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
+    /** Default node port. */
+    private static final int DEFAULT_NODE_PORT = 3344;
+
+    /** Value producer for table data, is used to create data and check it later. */
+    private static final IntFunction<String> VALUE_PRODUCER = i -> "val " + i;
+
+    /** Prefix for full table name. */
+    private static final String SCHEMA_PREFIX = "PUBLIC.";
+
+    /** Test table name. */
+    private static final String TABLE_NAME = "Table1";
+
+    /** Nodes bootstrap configuration pattern. */
+    private static final String NODE_BOOTSTRAP_CFG = "{\n"
+            + "  network.port: {},\n"
+            + "  network.nodeFinder.netClusterNodes: {}\n"
+            + "}";
+
+    /** Cluster nodes. */
+    private static final List<Ignite> CLUSTER_NODES = new ArrayList<>();
+
+    private static final List<String> CLUSTER_NODES_NAMES = new ArrayList<>();
+
+    /**
+     * Stops all started nodes.
+     */
+    @AfterEach
+    public void afterEach() throws Exception {
+        var closeables = new ArrayList<AutoCloseable>();
+
+        for (String name : CLUSTER_NODES_NAMES) {
+            if (name != null) {
+                closeables.add(() -> IgnitionManager.stop(name));
+            }
+        }
+
+        CLUSTER_NODES.clear();
+        CLUSTER_NODES_NAMES.clear();
+
+        IgniteUtils.closeAll(closeables);
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @param nodeName Node name.
+     * @param cfgString Configuration string.
+     * @param workDir Working directory.
+     * @return Created node instance.
+     */
+    private static IgniteImpl startNode(int idx, String nodeName, @Nullable String cfgString, Path workDir) {
+        assertTrue(CLUSTER_NODES.size() == idx || CLUSTER_NODES.get(idx) == null);
+
+        CLUSTER_NODES_NAMES.add(idx, nodeName);
+
+        CompletableFuture<Ignite> future = IgnitionManager.start(nodeName, cfgString, workDir.resolve(nodeName));
+
+        if (CLUSTER_NODES.isEmpty()) {
+            IgnitionManager.init(nodeName, List.of(nodeName), "cluster");
+        }
+
+        assertThat(future, willCompleteSuccessfully());
+
+        Ignite ignite = future.join();
+
+        CLUSTER_NODES.add(idx, ignite);
+
+        return (IgniteImpl) ignite;
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param testInfo Test info.
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @return Created node instance.
+     */
+    private IgniteImpl startNode(TestInfo testInfo, int idx) {
+        int port = DEFAULT_NODE_PORT + idx;
+        String nodeName = testNodeName(testInfo, port);
+        String cfgString = configurationString(idx, null, null);
+
+        return startNode(idx, nodeName, cfgString, workDir.resolve(nodeName));
+    }
+
+    /**
+     * Build a configuration string.
+     *
+     * @param idx Node index.
+     * @param cfg Optional configuration string.
+     * @param predefinedPort Predefined port.
+     * @return Configuration string.
+     */
+    private static String configurationString(int idx, @Nullable String cfg, @Nullable Integer predefinedPort) {
+        int port = predefinedPort == null ? DEFAULT_NODE_PORT + idx : predefinedPort;
+        int connectPort = predefinedPort == null ? DEFAULT_NODE_PORT : predefinedPort;
+        String connectAddr = "[\"localhost:" + connectPort + "\"]";

Review Comment:
   Practically the same but looks worse imho, because the semicolon should be escaped
   `String connectAddr = "[localhost\":\"" + DEFAULT_NODE_PORT + "]";`



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957427737


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);

Review Comment:
   What can I say, we must refactor the hell out of 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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958114339


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -157,6 +171,12 @@ public class RebalanceUtil {
     /** Key prefix for stable assignments. */
     public static final String STABLE_ASSIGNMENTS_PREFIX = "assignments.stable.";
 
+    /** Key prefix for switch reduce assignments. */
+    public static final String ASSIGNMENTS_SWITCH_REDUCE_PREFIX = "assignments.switch.reduce.";

Review Comment:
   We can do it later, because it is unrelated. It's just funny how we compare "writeObject" with native UOS, but then send these big ass strings over the network



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957293182


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/SimpleCondition.java:
##########
@@ -77,6 +77,8 @@ public long revision() {
          * @throws IllegalStateException In the case when the condition is already defined.
          */
         public SimpleCondition eq(long rev) {
+            assert rev > 0 : "Revision must be positive.";

Review Comment:
   1) It's basically the same check as on the metastorage's server side, see `RevisionCondition`'s constructor. 
   2) What do you mean? Something like `Requires#requireNonNull`?



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r956065558


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/SimpleCondition.java:
##########
@@ -77,6 +77,8 @@ public long revision() {
          * @throws IllegalStateException In the case when the condition is already defined.
          */
         public SimpleCondition eq(long rev) {
+            assert rev > 0 : "Revision must be positive.";

Review Comment:
   First of all, what's the point of this check?
   Second, why can't it be a normal argument validation?



##########
modules/network-annotation-processor/src/main/java/org/apache/ignite/internal/network/processor/messages/MessageImplGenerator.java:
##########
@@ -163,6 +166,16 @@ public TypeSpec generateMessageImpl(MessageClass message, TypeSpec builderInterf
 
         messageImpl.addMethod(groupTypeMethod);
 
+        MethodSpec toStringMethod = MethodSpec.methodBuilder("toString")
+                .addAnnotation(Override.class)
+                .addModifiers(Modifier.PUBLIC)
+                .returns(String.class)
+                .addStatement("return $T.toString($T.class, this)", S.class, messageImplClassName)
+                .build();
+
+        messageImpl.addMethod(toStringMethod);
+

Review Comment:
   Too many empty lines here



##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java:
##########
@@ -529,10 +532,12 @@ private final class Watcher extends Thread {
             /**
              * Constructor.
              *
+             * @param nodeId Node id.
              * @param cursor Watch event cursor.
              * @param lsnr   The listener which receives and handles watch updates.
              */
-            Watcher(Cursor<WatchEvent> cursor, WatchListener lsnr) {
+            Watcher(String nodeId, Cursor<WatchEvent> cursor, WatchListener lsnr) {
+                setName(nodeId);

Review Comment:
   Can you pass a good name that has a recognizable prefix?



##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java:
##########
@@ -679,15 +690,45 @@ private static boolean recoverable(Throwable t) {
         return t instanceof TimeoutException || t instanceof IOException;
     }
 
+    private Peer randomNode() {
+        return randomNode(null);
+    }
+
     /**
-     * @return Random node.
+     * Returns a random peer. Tries returning peer different from the peer passed as an argument.
+     * If peer is null, just returns a random peer.
+     *
+     * @param peer Previous peer.
+     * @return Random peer.
      */
-    private Peer randomNode() {
+    private Peer randomNode(Peer peer) {

Review Comment:
   I'd rename the parameter to "excludedPeer" or something,



##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java:
##########
@@ -603,7 +604,17 @@ else if (resp0.errorCode() == RaftError.EBUSY.getNumber() ||
                         resp0.errorCode() == (RaftError.EAGAIN.getNumber()) ||
                         resp0.errorCode() == (RaftError.ENOENT.getNumber())) { // Possibly a node has not been started.
                         executor.schedule(() -> {
-                            sendWithRetry(peer, req, stopTime, fut);
+                            Peer targetPeer = peer;
+
+                            if (resp0.errorCode() == (RaftError.ENOENT.getNumber())) {

Review Comment:
   ```suggestion
                               if (resp0.errorCode() == RaftError.ENOENT.getNumber()) {
   ```



##########
modules/network-annotation-processor/src/main/java/org/apache/ignite/internal/network/processor/messages/MessageImplGenerator.java:
##########
@@ -105,6 +107,7 @@ public TypeSpec generateMessageImpl(MessageClass message, TypeSpec builderInterf
             String getterName = getter.getSimpleName().toString();
 
             FieldSpec.Builder fieldBuilder = FieldSpec.builder(getterReturnType, getterName)
+                    .addAnnotation(IgniteToStringInclude.class)

Review Comment:
   We should think of marking some fields as sensitive or even exclude them. Just keep in mind



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java:
##########
@@ -0,0 +1,354 @@
+/*
+ * 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.runner.app;
+
+import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
+import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * These tests check in-memory node restart scenarios.
+ */
+public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
+    /** Default node port. */
+    private static final int DEFAULT_NODE_PORT = 3344;
+
+    /** Value producer for table data, is used to create data and check it later. */
+    private static final IntFunction<String> VALUE_PRODUCER = i -> "val " + i;
+
+    /** Prefix for full table name. */
+    private static final String SCHEMA_PREFIX = "PUBLIC.";
+
+    /** Test table name. */
+    private static final String TABLE_NAME = "Table1";
+
+    /** Nodes bootstrap configuration pattern. */
+    private static final String NODE_BOOTSTRAP_CFG = "{\n"
+            + "  network.port: {},\n"
+            + "  network.nodeFinder.netClusterNodes: {}\n"
+            + "}";
+
+    /** Cluster nodes. */
+    private static final List<Ignite> CLUSTER_NODES = new ArrayList<>();
+
+    private static final List<String> CLUSTER_NODES_NAMES = new ArrayList<>();
+
+    /**
+     * Stops all started nodes.
+     */
+    @AfterEach
+    public void afterEach() throws Exception {
+        var closeables = new ArrayList<AutoCloseable>();
+
+        for (String name : CLUSTER_NODES_NAMES) {
+            if (name != null) {
+                closeables.add(() -> IgnitionManager.stop(name));
+            }
+        }
+
+        CLUSTER_NODES.clear();
+        CLUSTER_NODES_NAMES.clear();
+
+        IgniteUtils.closeAll(closeables);
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @param nodeName Node name.
+     * @param cfgString Configuration string.
+     * @param workDir Working directory.
+     * @return Created node instance.
+     */
+    private static IgniteImpl startNode(int idx, String nodeName, @Nullable String cfgString, Path workDir) {
+        assertTrue(CLUSTER_NODES.size() == idx || CLUSTER_NODES.get(idx) == null);
+
+        CLUSTER_NODES_NAMES.add(idx, nodeName);
+
+        CompletableFuture<Ignite> future = IgnitionManager.start(nodeName, cfgString, workDir.resolve(nodeName));
+
+        if (CLUSTER_NODES.isEmpty()) {
+            IgnitionManager.init(nodeName, List.of(nodeName), "cluster");
+        }
+
+        assertThat(future, willCompleteSuccessfully());
+
+        Ignite ignite = future.join();
+
+        CLUSTER_NODES.add(idx, ignite);
+
+        return (IgniteImpl) ignite;
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param testInfo Test info.
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @return Created node instance.
+     */
+    private IgniteImpl startNode(TestInfo testInfo, int idx) {
+        int port = DEFAULT_NODE_PORT + idx;
+        String nodeName = testNodeName(testInfo, port);
+        String cfgString = configurationString(idx, null, null);
+
+        return startNode(idx, nodeName, cfgString, workDir.resolve(nodeName));
+    }
+
+    /**
+     * Build a configuration string.
+     *
+     * @param idx Node index.
+     * @param cfg Optional configuration string.
+     * @param predefinedPort Predefined port.
+     * @return Configuration string.
+     */
+    private static String configurationString(int idx, @Nullable String cfg, @Nullable Integer predefinedPort) {
+        int port = predefinedPort == null ? DEFAULT_NODE_PORT + idx : predefinedPort;
+        int connectPort = predefinedPort == null ? DEFAULT_NODE_PORT : predefinedPort;

Review Comment:
   What's going on here? All nodes will have the same connect port, seems weird.



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {

Review Comment:
   Can this be a lambda?
   Why do we need to handle all group messages instead of the individual handler for "HasDataRequest"?



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java:
##########
@@ -0,0 +1,354 @@
+/*
+ * 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.runner.app;
+
+import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
+import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * These tests check in-memory node restart scenarios.
+ */
+public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
+    /** Default node port. */
+    private static final int DEFAULT_NODE_PORT = 3344;
+
+    /** Value producer for table data, is used to create data and check it later. */
+    private static final IntFunction<String> VALUE_PRODUCER = i -> "val " + i;
+
+    /** Prefix for full table name. */
+    private static final String SCHEMA_PREFIX = "PUBLIC.";
+
+    /** Test table name. */
+    private static final String TABLE_NAME = "Table1";
+
+    /** Nodes bootstrap configuration pattern. */
+    private static final String NODE_BOOTSTRAP_CFG = "{\n"
+            + "  network.port: {},\n"
+            + "  network.nodeFinder.netClusterNodes: {}\n"
+            + "}";
+
+    /** Cluster nodes. */
+    private static final List<Ignite> CLUSTER_NODES = new ArrayList<>();
+
+    private static final List<String> CLUSTER_NODES_NAMES = new ArrayList<>();
+
+    /**
+     * Stops all started nodes.
+     */
+    @AfterEach
+    public void afterEach() throws Exception {
+        var closeables = new ArrayList<AutoCloseable>();
+
+        for (String name : CLUSTER_NODES_NAMES) {
+            if (name != null) {
+                closeables.add(() -> IgnitionManager.stop(name));
+            }
+        }
+
+        CLUSTER_NODES.clear();
+        CLUSTER_NODES_NAMES.clear();
+
+        IgniteUtils.closeAll(closeables);
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @param nodeName Node name.
+     * @param cfgString Configuration string.
+     * @param workDir Working directory.
+     * @return Created node instance.
+     */
+    private static IgniteImpl startNode(int idx, String nodeName, @Nullable String cfgString, Path workDir) {
+        assertTrue(CLUSTER_NODES.size() == idx || CLUSTER_NODES.get(idx) == null);
+
+        CLUSTER_NODES_NAMES.add(idx, nodeName);
+
+        CompletableFuture<Ignite> future = IgnitionManager.start(nodeName, cfgString, workDir.resolve(nodeName));
+
+        if (CLUSTER_NODES.isEmpty()) {
+            IgnitionManager.init(nodeName, List.of(nodeName), "cluster");
+        }
+
+        assertThat(future, willCompleteSuccessfully());
+
+        Ignite ignite = future.join();
+
+        CLUSTER_NODES.add(idx, ignite);
+
+        return (IgniteImpl) ignite;
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param testInfo Test info.
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @return Created node instance.
+     */
+    private IgniteImpl startNode(TestInfo testInfo, int idx) {
+        int port = DEFAULT_NODE_PORT + idx;
+        String nodeName = testNodeName(testInfo, port);
+        String cfgString = configurationString(idx, null, null);
+
+        return startNode(idx, nodeName, cfgString, workDir.resolve(nodeName));
+    }
+
+    /**
+     * Build a configuration string.
+     *
+     * @param idx Node index.
+     * @param cfg Optional configuration string.
+     * @param predefinedPort Predefined port.
+     * @return Configuration string.
+     */
+    private static String configurationString(int idx, @Nullable String cfg, @Nullable Integer predefinedPort) {
+        int port = predefinedPort == null ? DEFAULT_NODE_PORT + idx : predefinedPort;
+        int connectPort = predefinedPort == null ? DEFAULT_NODE_PORT : predefinedPort;
+        String connectAddr = "[\"localhost:" + connectPort + "\"]";

Review Comment:
   You don't have to put quotes in HOCON. This string can be simplified



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
+
+                                        if (majorityAvailable) {
+                                            String partitionId = partitionRaftGroupName(tblId, partId);
+                                            RebalanceUtil.startPeerRemoval(partitionId, localMember, metaStorageMgr);
+                                            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.";
+                                            throw new IgniteInternalException(msg);
+                                        }
+                                    });
+                                } else {
+                                    fut = CompletableFuture.completedFuture(true);
                                 }
+
+                                return fut.thenComposeAsync(startGroup -> {

Review Comment:
   Why do we need to do supplyAsync inside of the other sypplyAsync? Can we have a normal composition here? "fut" object can just be returned from this closure.
   BTW, this way we will avoid a name like `fut`!



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java:
##########
@@ -0,0 +1,354 @@
+/*
+ * 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.runner.app;
+
+import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
+import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * These tests check in-memory node restart scenarios.
+ */
+public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
+    /** Default node port. */
+    private static final int DEFAULT_NODE_PORT = 3344;
+
+    /** Value producer for table data, is used to create data and check it later. */
+    private static final IntFunction<String> VALUE_PRODUCER = i -> "val " + i;
+
+    /** Prefix for full table name. */
+    private static final String SCHEMA_PREFIX = "PUBLIC.";
+
+    /** Test table name. */
+    private static final String TABLE_NAME = "Table1";
+
+    /** Nodes bootstrap configuration pattern. */
+    private static final String NODE_BOOTSTRAP_CFG = "{\n"
+            + "  network.port: {},\n"
+            + "  network.nodeFinder.netClusterNodes: {}\n"
+            + "}";
+
+    /** Cluster nodes. */
+    private static final List<Ignite> CLUSTER_NODES = new ArrayList<>();
+
+    private static final List<String> CLUSTER_NODES_NAMES = new ArrayList<>();
+
+    /**
+     * Stops all started nodes.
+     */
+    @AfterEach
+    public void afterEach() throws Exception {
+        var closeables = new ArrayList<AutoCloseable>();
+
+        for (String name : CLUSTER_NODES_NAMES) {
+            if (name != null) {
+                closeables.add(() -> IgnitionManager.stop(name));
+            }
+        }
+
+        CLUSTER_NODES.clear();
+        CLUSTER_NODES_NAMES.clear();
+
+        IgniteUtils.closeAll(closeables);
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @param nodeName Node name.
+     * @param cfgString Configuration string.
+     * @param workDir Working directory.
+     * @return Created node instance.
+     */
+    private static IgniteImpl startNode(int idx, String nodeName, @Nullable String cfgString, Path workDir) {
+        assertTrue(CLUSTER_NODES.size() == idx || CLUSTER_NODES.get(idx) == null);
+
+        CLUSTER_NODES_NAMES.add(idx, nodeName);
+
+        CompletableFuture<Ignite> future = IgnitionManager.start(nodeName, cfgString, workDir.resolve(nodeName));
+
+        if (CLUSTER_NODES.isEmpty()) {
+            IgnitionManager.init(nodeName, List.of(nodeName), "cluster");
+        }
+
+        assertThat(future, willCompleteSuccessfully());
+
+        Ignite ignite = future.join();
+
+        CLUSTER_NODES.add(idx, ignite);
+
+        return (IgniteImpl) ignite;
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param testInfo Test info.
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @return Created node instance.
+     */
+    private IgniteImpl startNode(TestInfo testInfo, int idx) {
+        int port = DEFAULT_NODE_PORT + idx;
+        String nodeName = testNodeName(testInfo, port);
+        String cfgString = configurationString(idx, null, null);
+
+        return startNode(idx, nodeName, cfgString, workDir.resolve(nodeName));
+    }
+
+    /**
+     * Build a configuration string.
+     *
+     * @param idx Node index.
+     * @param cfg Optional configuration string.
+     * @param predefinedPort Predefined port.
+     * @return Configuration string.
+     */
+    private static String configurationString(int idx, @Nullable String cfg, @Nullable Integer predefinedPort) {
+        int port = predefinedPort == null ? DEFAULT_NODE_PORT + idx : predefinedPort;
+        int connectPort = predefinedPort == null ? DEFAULT_NODE_PORT : predefinedPort;
+        String connectAddr = "[\"localhost:" + connectPort + "\"]";
+
+        return cfg == null ? IgniteStringFormatter.format(NODE_BOOTSTRAP_CFG, port, connectAddr) : cfg;
+    }
+
+    /**
+     * Stop the node with given index.
+     *
+     * @param idx Node index.
+     */
+    private static void stopNode(int idx) {

Review Comment:
   In "afterAll" you clear static collections and then stop nodes, but here it's the opposite. Why?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);
+
+                    if (table != null) {
+                        MvTableStorage storage = table.internalTable().storage();
+
+                        MvPartitionStorage mvPartition = storage.getMvPartition(partitionId);
+
+                        if (mvPartition != null) {
+                            // If applied index of a storage is greater than 0,
+                            // then there is data
+                            contains = mvPartition.lastAppliedIndex() > 0;

Review Comment:
   This code assumes that node recovery is completed, right? I think that this part deserves a comment.
   If don't know what I'm talking about or this code is expected to work even before recovery is complete, then there's a problem here, probably



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
+
+                                        if (majorityAvailable) {
+                                            String partitionId = partitionRaftGroupName(tblId, partId);

Review Comment:
   This is a raft group name, not a partition id



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
+
+                                        if (majorityAvailable) {
+                                            String partitionId = partitionRaftGroupName(tblId, partId);
+                                            RebalanceUtil.startPeerRemoval(partitionId, localMember, metaStorageMgr);
+                                            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.";

Review Comment:
   I see a lack of empty lines in your code



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -606,6 +708,32 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
         CompletableFuture.allOf(futures).join();
     }
 
+    /**
+     * Calculates the quantity of the data nodes for the partition of the table.
+     *
+     * @param tblId Table id.
+     * @param partId Partition id.
+     * @param partAssignments Partition assignments.
+     * @return A future that will hold the quantity of data nodes.
+     */
+    private CompletableFuture<Long> queryDataNodesCount(UUID tblId, int partId, List<ClusterNode> partAssignments) {

Review Comment:
   `Long`. Really?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/RebalanceRaftGroupEventsListener.java:
##########
@@ -252,92 +294,189 @@ private void scheduleChangePeers(List<PeerId> peers, long term) {
      */
     private void doOnNewPeersConfigurationApplied(List<PeerId> peers) {
         try {
-            Map<ByteArray, Entry> keys = metaStorageMgr.getAll(
+            ByteArray pendingPartAssignmentsKey = pendingPartAssignmentsKey(partId);
+            ByteArray stablePartAssignmentsKey = stablePartAssignmentsKey(partId);
+            ByteArray plannedPartAssignmentsKey = plannedPartAssignmentsKey(partId);
+            ByteArray switchReduceKey = switchReduceKey(partId);
+            ByteArray switchAppendKey = switchAppendKey(partId);
+
+            Map<ByteArray, Entry> values = metaStorageMgr.getAll(
                     Set.of(
-                            plannedPartAssignmentsKey(partId),
-                            pendingPartAssignmentsKey(partId),
-                            stablePartAssignmentsKey(partId))).get();
+                            plannedPartAssignmentsKey,
+                            pendingPartAssignmentsKey,
+                            stablePartAssignmentsKey,
+                            switchReduceKey,
+                            switchAppendKey
+                    )
+            ).get();

Review Comment:
   This deserves a TODO. Another synchronous wait. You're not the author, right?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1562,6 +1697,40 @@ public void onError(@NotNull Throwable e) {
                 LOG.warn("Unable to process stable assignments event", e);
             }
         });
+
+        metaStorageMgr.registerWatchByPrefix(ByteArray.fromString(ASSIGNMENTS_SWITCH_REDUCE_PREFIX), new WatchListener() {
+            @Override
+            public boolean onUpdate(@NotNull WatchEvent evt) {
+                ByteArray key = evt.entryEvent().newEntry().key();
+
+                int partitionNumber = extractPartitionNumber(key);
+                UUID tblId = extractTableId(key, ASSIGNMENTS_SWITCH_REDUCE_PREFIX);
+                String partitionId = partitionRaftGroupName(tblId, partitionNumber);
+
+                TableImpl tbl = tablesByIdVv.latest().get(tblId);
+                ExtendedTableConfiguration tblCfg = (ExtendedTableConfiguration) tablesCfg.tables().get(tbl.name());

Review Comment:
   I wonder if there's a race possible with configuration update. Why getting the latest value is ok?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/RebalanceRaftGroupEventsListener.java:
##########
@@ -252,92 +294,189 @@ private void scheduleChangePeers(List<PeerId> peers, long term) {
      */
     private void doOnNewPeersConfigurationApplied(List<PeerId> peers) {
         try {
-            Map<ByteArray, Entry> keys = metaStorageMgr.getAll(
+            ByteArray pendingPartAssignmentsKey = pendingPartAssignmentsKey(partId);
+            ByteArray stablePartAssignmentsKey = stablePartAssignmentsKey(partId);
+            ByteArray plannedPartAssignmentsKey = plannedPartAssignmentsKey(partId);
+            ByteArray switchReduceKey = switchReduceKey(partId);
+            ByteArray switchAppendKey = switchAppendKey(partId);
+
+            Map<ByteArray, Entry> values = metaStorageMgr.getAll(
                     Set.of(
-                            plannedPartAssignmentsKey(partId),
-                            pendingPartAssignmentsKey(partId),
-                            stablePartAssignmentsKey(partId))).get();
+                            plannedPartAssignmentsKey,
+                            pendingPartAssignmentsKey,
+                            stablePartAssignmentsKey,
+                            switchReduceKey,
+                            switchAppendKey
+                    )
+            ).get();
 
-            Entry plannedEntry = keys.get(plannedPartAssignmentsKey(partId));
+            Entry stableEntry = values.get(stablePartAssignmentsKey);
+            Entry pendingEntry = values.get(pendingPartAssignmentsKey);
+            Entry plannedEntry = values.get(plannedPartAssignmentsKey);
+            Entry switchReduceEntry = values.get(switchReduceKey);
+            Entry switchAppendEntry = values.get(switchAppendKey);
 
-            List<ClusterNode> appliedPeers = resolveClusterNodes(peers,
-                    keys.get(pendingPartAssignmentsKey(partId)).value(), keys.get(stablePartAssignmentsKey(partId)).value());
+            List<ClusterNode> calculatedAssignments = calculateAssignmentsFn.apply(tblConfiguration, partNum);
 
-            tblConfiguration.change(ch -> {
-                List<List<ClusterNode>> assignments =
-                        (List<List<ClusterNode>>) ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
-                assignments.set(partNum, appliedPeers);
-                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
-            }).get();
-
-            if (plannedEntry.value() != null) {
-                if (!metaStorageMgr.invoke(If.iif(
-                        revision(plannedPartAssignmentsKey(partId)).eq(plannedEntry.revision()),
-                        ops(
-                                put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                put(pendingPartAssignmentsKey(partId), plannedEntry.value()),
-                                remove(plannedPartAssignmentsKey(partId)))
-                                .yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            List<ClusterNode> stable = resolveClusterNodes(peers, pendingEntry.value(), stableEntry.value());
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            List<ClusterNode> retrievedSwitchReduce = readClusterNodes(switchReduceEntry);
+            List<ClusterNode> retrievedSwitchAppend = readClusterNodes(switchAppendEntry);
+            List<ClusterNode> retrievedStable = readClusterNodes(stableEntry);
 
-                LOG.info("Rebalance finished. Going to schedule next rebalance [partition={}, table={}, appliedPeers={}, plannedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers, ByteUtils.fromBytes(plannedEntry.value()));
-            } else {
-                if (!metaStorageMgr.invoke(If.iif(
-                        notExists(plannedPartAssignmentsKey(partId)),
-                        ops(put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                remove(pendingPartAssignmentsKey(partId))).yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            // Were reduced
+            List<ClusterNode> reducedNodes = subtract(retrievedSwitchReduce, stable);
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            // Were added
+            List<ClusterNode> addedNodes = subtract(stable, retrievedStable);
 
-                LOG.info("Rebalance finished [partition={}, table={}, appliedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers);
-            }
+            // For further reduction
+            List<ClusterNode> calculatedSwitchReduce = subtract(retrievedSwitchReduce, reducedNodes);
 
-            rebalanceAttempts.set(0);
-        } catch (InterruptedException | ExecutionException e) {
-            // TODO: IGNITE-14693
-            LOG.warn("Unable to commit partition configuration to metastore [table = {}, partition = {}]",
-                    e, tblConfiguration.name(), partNum);
-        }
-    }
+            // For further addition
+            List<ClusterNode> calculatedSwitchAppend = union(retrievedSwitchAppend, reducedNodes);
+            calculatedSwitchAppend = subtract(calculatedSwitchAppend, addedNodes);
+            calculatedSwitchAppend = intersect(calculatedAssignments, calculatedSwitchAppend);
 
-    private static List<ClusterNode> resolveClusterNodes(
-            List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
-        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+            var calculatedPendingReduction = subtract(stable, retrievedSwitchReduce);

Review Comment:
   What's up with all these "var"? We shouldn't use them here



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -157,6 +171,12 @@ public class RebalanceUtil {
     /** Key prefix for stable assignments. */
     public static final String STABLE_ASSIGNMENTS_PREFIX = "assignments.stable.";
 
+    /** Key prefix for switch reduce assignments. */
+    public static final String ASSIGNMENTS_SWITCH_REDUCE_PREFIX = "assignments.switch.reduce.";

Review Comment:
   I guess it's not my business, but these names are too long. We will pollute metastorage with key names, it's horrible. I'm not laming you, of course



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1562,6 +1697,40 @@ public void onError(@NotNull Throwable e) {
                 LOG.warn("Unable to process stable assignments event", e);
             }
         });
+
+        metaStorageMgr.registerWatchByPrefix(ByteArray.fromString(ASSIGNMENTS_SWITCH_REDUCE_PREFIX), new WatchListener() {
+            @Override
+            public boolean onUpdate(@NotNull WatchEvent evt) {
+                ByteArray key = evt.entryEvent().newEntry().key();
+
+                int partitionNumber = extractPartitionNumber(key);
+                UUID tblId = extractTableId(key, ASSIGNMENTS_SWITCH_REDUCE_PREFIX);
+                String partitionId = partitionRaftGroupName(tblId, partitionNumber);
+
+                TableImpl tbl = tablesByIdVv.latest().get(tblId);
+                ExtendedTableConfiguration tblCfg = (ExtendedTableConfiguration) tablesCfg.tables().get(tbl.name());

Review Comment:
   What's the point of casting it to extended tale configuration if you don't use any of the extension properties?



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.

Review Comment:
   "has been", maybe?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/RebalanceRaftGroupEventsListener.java:
##########
@@ -252,92 +294,189 @@ private void scheduleChangePeers(List<PeerId> peers, long term) {
      */
     private void doOnNewPeersConfigurationApplied(List<PeerId> peers) {
         try {
-            Map<ByteArray, Entry> keys = metaStorageMgr.getAll(
+            ByteArray pendingPartAssignmentsKey = pendingPartAssignmentsKey(partId);
+            ByteArray stablePartAssignmentsKey = stablePartAssignmentsKey(partId);
+            ByteArray plannedPartAssignmentsKey = plannedPartAssignmentsKey(partId);
+            ByteArray switchReduceKey = switchReduceKey(partId);
+            ByteArray switchAppendKey = switchAppendKey(partId);
+
+            Map<ByteArray, Entry> values = metaStorageMgr.getAll(
                     Set.of(
-                            plannedPartAssignmentsKey(partId),
-                            pendingPartAssignmentsKey(partId),
-                            stablePartAssignmentsKey(partId))).get();
+                            plannedPartAssignmentsKey,
+                            pendingPartAssignmentsKey,
+                            stablePartAssignmentsKey,
+                            switchReduceKey,
+                            switchAppendKey
+                    )
+            ).get();
 
-            Entry plannedEntry = keys.get(plannedPartAssignmentsKey(partId));
+            Entry stableEntry = values.get(stablePartAssignmentsKey);
+            Entry pendingEntry = values.get(pendingPartAssignmentsKey);
+            Entry plannedEntry = values.get(plannedPartAssignmentsKey);
+            Entry switchReduceEntry = values.get(switchReduceKey);
+            Entry switchAppendEntry = values.get(switchAppendKey);
 
-            List<ClusterNode> appliedPeers = resolveClusterNodes(peers,
-                    keys.get(pendingPartAssignmentsKey(partId)).value(), keys.get(stablePartAssignmentsKey(partId)).value());
+            List<ClusterNode> calculatedAssignments = calculateAssignmentsFn.apply(tblConfiguration, partNum);
 
-            tblConfiguration.change(ch -> {
-                List<List<ClusterNode>> assignments =
-                        (List<List<ClusterNode>>) ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
-                assignments.set(partNum, appliedPeers);
-                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
-            }).get();
-
-            if (plannedEntry.value() != null) {
-                if (!metaStorageMgr.invoke(If.iif(
-                        revision(plannedPartAssignmentsKey(partId)).eq(plannedEntry.revision()),
-                        ops(
-                                put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                put(pendingPartAssignmentsKey(partId), plannedEntry.value()),
-                                remove(plannedPartAssignmentsKey(partId)))
-                                .yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            List<ClusterNode> stable = resolveClusterNodes(peers, pendingEntry.value(), stableEntry.value());
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            List<ClusterNode> retrievedSwitchReduce = readClusterNodes(switchReduceEntry);
+            List<ClusterNode> retrievedSwitchAppend = readClusterNodes(switchAppendEntry);
+            List<ClusterNode> retrievedStable = readClusterNodes(stableEntry);
 
-                LOG.info("Rebalance finished. Going to schedule next rebalance [partition={}, table={}, appliedPeers={}, plannedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers, ByteUtils.fromBytes(plannedEntry.value()));
-            } else {
-                if (!metaStorageMgr.invoke(If.iif(
-                        notExists(plannedPartAssignmentsKey(partId)),
-                        ops(put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                remove(pendingPartAssignmentsKey(partId))).yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            // Were reduced
+            List<ClusterNode> reducedNodes = subtract(retrievedSwitchReduce, stable);
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            // Were added
+            List<ClusterNode> addedNodes = subtract(stable, retrievedStable);
 
-                LOG.info("Rebalance finished [partition={}, table={}, appliedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers);
-            }
+            // For further reduction
+            List<ClusterNode> calculatedSwitchReduce = subtract(retrievedSwitchReduce, reducedNodes);
 
-            rebalanceAttempts.set(0);
-        } catch (InterruptedException | ExecutionException e) {
-            // TODO: IGNITE-14693
-            LOG.warn("Unable to commit partition configuration to metastore [table = {}, partition = {}]",
-                    e, tblConfiguration.name(), partNum);
-        }
-    }
+            // For further addition
+            List<ClusterNode> calculatedSwitchAppend = union(retrievedSwitchAppend, reducedNodes);
+            calculatedSwitchAppend = subtract(calculatedSwitchAppend, addedNodes);
+            calculatedSwitchAppend = intersect(calculatedAssignments, calculatedSwitchAppend);
 
-    private static List<ClusterNode> resolveClusterNodes(
-            List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
-        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+            var calculatedPendingReduction = subtract(stable, retrievedSwitchReduce);
 
-        if (pendingAssignments != null) {
-            ((List<ClusterNode>) ByteUtils.fromBytes(pendingAssignments)).forEach(n -> resolveRegistry.put(n.address(), n));
-        }
+            var calculatedPendingAddition = union(stable, reducedNodes);
+            calculatedPendingAddition = intersect(calculatedAssignments, calculatedPendingAddition);
 
-        if (stableAssignments != null) {
-            ((List<ClusterNode>) ByteUtils.fromBytes(stableAssignments)).forEach(n -> resolveRegistry.put(n.address(), n));
-        }
+            // eq(revision(assignments.stable), retrievedAssignmentsStable.revision)
+            SimpleCondition con1 = stableEntry.empty()
+                    ? notExists(stablePartAssignmentsKey) :
+                    revision(stablePartAssignmentsKey).eq(stableEntry.revision());
+
+            // eq(revision(assignments.pending), retrievedAssignmentsPending.revision)
+            SimpleCondition con2 = revision(pendingPartAssignmentsKey).eq(pendingEntry.revision());
 
-        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+            // eq(revision(assignments.switch.reduce), retrievedAssignmentsSwitchReduce.revision)
+            SimpleCondition con3 = switchReduceEntry.empty()
+                    ? notExists(switchReduceKey) : revision(switchReduceKey).eq(switchReduceEntry.revision());
 
-        for (PeerId p : peers) {
-            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+            // eq(revision(assignments.switch.append), retrievedAssignmentsSwitchAppend.revision)
+            SimpleCondition con4 = switchAppendEntry.empty()
+                    ? notExists(switchAppendKey) : revision(switchAppendKey).eq(switchAppendEntry.revision());
 
-            if (resolveRegistry.containsKey(addr)) {
-                resolvedNodes.add(resolveRegistry.get(addr));
+            Condition retryPreconditions = and(con1, and(con2, and(con3, con4)));
+
+            tblConfiguration.change(ch -> {
+                List<List<ClusterNode>> assignments = ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
+                assignments.set(partNum, stable);
+                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
+            }).get(10, TimeUnit.SECONDS);
+
+            Update successCase;
+            Update failCase;
+
+            byte[] stableByteArray = ByteUtils.toBytes(stable);
+            byte[] additionByteArray = ByteUtils.toBytes(calculatedPendingAddition);
+            byte[] reductionByteArray = ByteUtils.toBytes(calculatedPendingReduction);
+            byte[] switchReduceByteArray = ByteUtils.toBytes(calculatedSwitchReduce);
+            byte[] switchAppendByteArray = ByteUtils.toBytes(calculatedSwitchAppend);
+
+            if (!calculatedSwitchAppend.isEmpty()) {
+                successCase = ops(
+                        put(stablePartAssignmentsKey, stableByteArray),
+                        put(pendingPartAssignmentsKey, additionByteArray),
+                        put(switchReduceKey, switchReduceByteArray),
+                        put(switchAppendKey, switchAppendByteArray)
+                ).yield(SWITCH_APPEND_SUCCESS);
+                failCase = ops().yield(SWITCH_APPEND_FAIL);
+            } else if (!calculatedSwitchReduce.isEmpty()) {
+                successCase = ops(
+                        put(stablePartAssignmentsKey, stableByteArray),
+                        put(pendingPartAssignmentsKey, reductionByteArray),
+                        put(switchReduceKey, switchReduceByteArray),
+                        put(switchAppendKey, switchAppendByteArray)
+                ).yield(SWITCH_REDUCE_SUCCESS);
+                failCase = ops().yield(SWITCH_REDUCE_FAIL);
             } else {
-                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+                Condition con5;

Review Comment:
   Variable names look odd



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;

Review Comment:
   Why can't it be one huge "if" with `prevValue == null` condition? Is it really simpler with a bunch of smaller ifs and extra variables?



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);

Review Comment:
   Same here. Assignments calculation is a computationally complicated process and it might use a lot of memory. There has to be a workaround here.



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {

Review Comment:
   Is this a partition id or an id of corresponding raft group?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/RebalanceRaftGroupEventsListener.java:
##########
@@ -252,92 +294,189 @@ private void scheduleChangePeers(List<PeerId> peers, long term) {
      */
     private void doOnNewPeersConfigurationApplied(List<PeerId> peers) {
         try {
-            Map<ByteArray, Entry> keys = metaStorageMgr.getAll(
+            ByteArray pendingPartAssignmentsKey = pendingPartAssignmentsKey(partId);
+            ByteArray stablePartAssignmentsKey = stablePartAssignmentsKey(partId);
+            ByteArray plannedPartAssignmentsKey = plannedPartAssignmentsKey(partId);
+            ByteArray switchReduceKey = switchReduceKey(partId);
+            ByteArray switchAppendKey = switchAppendKey(partId);
+
+            Map<ByteArray, Entry> values = metaStorageMgr.getAll(
                     Set.of(
-                            plannedPartAssignmentsKey(partId),
-                            pendingPartAssignmentsKey(partId),
-                            stablePartAssignmentsKey(partId))).get();
+                            plannedPartAssignmentsKey,
+                            pendingPartAssignmentsKey,
+                            stablePartAssignmentsKey,
+                            switchReduceKey,
+                            switchAppendKey
+                    )
+            ).get();
 
-            Entry plannedEntry = keys.get(plannedPartAssignmentsKey(partId));
+            Entry stableEntry = values.get(stablePartAssignmentsKey);
+            Entry pendingEntry = values.get(pendingPartAssignmentsKey);
+            Entry plannedEntry = values.get(plannedPartAssignmentsKey);
+            Entry switchReduceEntry = values.get(switchReduceKey);
+            Entry switchAppendEntry = values.get(switchAppendKey);
 
-            List<ClusterNode> appliedPeers = resolveClusterNodes(peers,
-                    keys.get(pendingPartAssignmentsKey(partId)).value(), keys.get(stablePartAssignmentsKey(partId)).value());
+            List<ClusterNode> calculatedAssignments = calculateAssignmentsFn.apply(tblConfiguration, partNum);
 
-            tblConfiguration.change(ch -> {
-                List<List<ClusterNode>> assignments =
-                        (List<List<ClusterNode>>) ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
-                assignments.set(partNum, appliedPeers);
-                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
-            }).get();
-
-            if (plannedEntry.value() != null) {
-                if (!metaStorageMgr.invoke(If.iif(
-                        revision(plannedPartAssignmentsKey(partId)).eq(plannedEntry.revision()),
-                        ops(
-                                put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                put(pendingPartAssignmentsKey(partId), plannedEntry.value()),
-                                remove(plannedPartAssignmentsKey(partId)))
-                                .yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            List<ClusterNode> stable = resolveClusterNodes(peers, pendingEntry.value(), stableEntry.value());
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            List<ClusterNode> retrievedSwitchReduce = readClusterNodes(switchReduceEntry);
+            List<ClusterNode> retrievedSwitchAppend = readClusterNodes(switchAppendEntry);
+            List<ClusterNode> retrievedStable = readClusterNodes(stableEntry);
 
-                LOG.info("Rebalance finished. Going to schedule next rebalance [partition={}, table={}, appliedPeers={}, plannedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers, ByteUtils.fromBytes(plannedEntry.value()));
-            } else {
-                if (!metaStorageMgr.invoke(If.iif(
-                        notExists(plannedPartAssignmentsKey(partId)),
-                        ops(put(stablePartAssignmentsKey(partId), ByteUtils.toBytes(appliedPeers)),
-                                remove(pendingPartAssignmentsKey(partId))).yield(true),
-                        ops().yield(false))).get().getAsBoolean()) {
-                    LOG.info("Planned key changed while trying to update rebalance information. Going to retry"
-                                    + " [key={}, partition={}, table={}, appliedPeers={}]",
-                            plannedPartAssignmentsKey(partId), partNum, tblConfiguration.name(), appliedPeers);
+            // Were reduced
+            List<ClusterNode> reducedNodes = subtract(retrievedSwitchReduce, stable);
 
-                    doOnNewPeersConfigurationApplied(peers);
-                }
+            // Were added
+            List<ClusterNode> addedNodes = subtract(stable, retrievedStable);
 
-                LOG.info("Rebalance finished [partition={}, table={}, appliedPeers={}]",
-                        partNum, tblConfiguration.name().value(), appliedPeers);
-            }
+            // For further reduction
+            List<ClusterNode> calculatedSwitchReduce = subtract(retrievedSwitchReduce, reducedNodes);
 
-            rebalanceAttempts.set(0);
-        } catch (InterruptedException | ExecutionException e) {
-            // TODO: IGNITE-14693
-            LOG.warn("Unable to commit partition configuration to metastore [table = {}, partition = {}]",
-                    e, tblConfiguration.name(), partNum);
-        }
-    }
+            // For further addition
+            List<ClusterNode> calculatedSwitchAppend = union(retrievedSwitchAppend, reducedNodes);
+            calculatedSwitchAppend = subtract(calculatedSwitchAppend, addedNodes);
+            calculatedSwitchAppend = intersect(calculatedAssignments, calculatedSwitchAppend);
 
-    private static List<ClusterNode> resolveClusterNodes(
-            List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
-        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+            var calculatedPendingReduction = subtract(stable, retrievedSwitchReduce);
 
-        if (pendingAssignments != null) {
-            ((List<ClusterNode>) ByteUtils.fromBytes(pendingAssignments)).forEach(n -> resolveRegistry.put(n.address(), n));
-        }
+            var calculatedPendingAddition = union(stable, reducedNodes);
+            calculatedPendingAddition = intersect(calculatedAssignments, calculatedPendingAddition);
 
-        if (stableAssignments != null) {
-            ((List<ClusterNode>) ByteUtils.fromBytes(stableAssignments)).forEach(n -> resolveRegistry.put(n.address(), n));
-        }
+            // eq(revision(assignments.stable), retrievedAssignmentsStable.revision)
+            SimpleCondition con1 = stableEntry.empty()
+                    ? notExists(stablePartAssignmentsKey) :
+                    revision(stablePartAssignmentsKey).eq(stableEntry.revision());
+
+            // eq(revision(assignments.pending), retrievedAssignmentsPending.revision)
+            SimpleCondition con2 = revision(pendingPartAssignmentsKey).eq(pendingEntry.revision());
 
-        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+            // eq(revision(assignments.switch.reduce), retrievedAssignmentsSwitchReduce.revision)
+            SimpleCondition con3 = switchReduceEntry.empty()
+                    ? notExists(switchReduceKey) : revision(switchReduceKey).eq(switchReduceEntry.revision());
 
-        for (PeerId p : peers) {
-            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+            // eq(revision(assignments.switch.append), retrievedAssignmentsSwitchAppend.revision)
+            SimpleCondition con4 = switchAppendEntry.empty()
+                    ? notExists(switchAppendKey) : revision(switchAppendKey).eq(switchAppendEntry.revision());
 
-            if (resolveRegistry.containsKey(addr)) {
-                resolvedNodes.add(resolveRegistry.get(addr));
+            Condition retryPreconditions = and(con1, and(con2, and(con3, con4)));
+
+            tblConfiguration.change(ch -> {
+                List<List<ClusterNode>> assignments = ByteUtils.fromBytes(((ExtendedTableChange) ch).assignments());
+                assignments.set(partNum, stable);
+                ((ExtendedTableChange) ch).changeAssignments(ByteUtils.toBytes(assignments));
+            }).get(10, TimeUnit.SECONDS);
+
+            Update successCase;
+            Update failCase;
+
+            byte[] stableByteArray = ByteUtils.toBytes(stable);
+            byte[] additionByteArray = ByteUtils.toBytes(calculatedPendingAddition);
+            byte[] reductionByteArray = ByteUtils.toBytes(calculatedPendingReduction);
+            byte[] switchReduceByteArray = ByteUtils.toBytes(calculatedSwitchReduce);
+            byte[] switchAppendByteArray = ByteUtils.toBytes(calculatedSwitchAppend);
+
+            if (!calculatedSwitchAppend.isEmpty()) {
+                successCase = ops(
+                        put(stablePartAssignmentsKey, stableByteArray),
+                        put(pendingPartAssignmentsKey, additionByteArray),
+                        put(switchReduceKey, switchReduceByteArray),
+                        put(switchAppendKey, switchAppendByteArray)
+                ).yield(SWITCH_APPEND_SUCCESS);
+                failCase = ops().yield(SWITCH_APPEND_FAIL);
+            } else if (!calculatedSwitchReduce.isEmpty()) {
+                successCase = ops(
+                        put(stablePartAssignmentsKey, stableByteArray),
+                        put(pendingPartAssignmentsKey, reductionByteArray),
+                        put(switchReduceKey, switchReduceByteArray),
+                        put(switchAppendKey, switchAppendByteArray)
+                ).yield(SWITCH_REDUCE_SUCCESS);
+                failCase = ops().yield(SWITCH_REDUCE_FAIL);
             } else {
-                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+                Condition con5;
+                if (plannedEntry.value() != null) {
+                    // eq(revision(partition.assignments.planned), plannedEntry.revision)
+                    con5 = revision(plannedPartAssignmentsKey).eq(plannedEntry.revision());
+
+                    successCase = ops(
+                            put(stablePartAssignmentsKey, ByteUtils.toBytes(stable)),
+                            put(pendingPartAssignmentsKey, plannedEntry.value()),
+                            remove(plannedPartAssignmentsKey)
+                    ).yield(SCHEDULE_PENDING_REBALANCE_SUCCESS);
+
+                    failCase = ops().yield(SCHEDULE_PENDING_REBALANCE_FAIL);
+                } else {
+                    // notExists(partition.assignments.planned)
+                    con5 = notExists(plannedPartAssignmentsKey);
+
+                    successCase = ops(
+                            put(stablePartAssignmentsKey, ByteUtils.toBytes(stable)),
+                            remove(pendingPartAssignmentsKey)
+                    ).yield(FINISH_REBALANCE_SUCCESS);
+
+                    failCase = ops().yield(FINISH_REBALANCE_FAIL);
+                }
+
+                retryPreconditions = and(retryPreconditions, con5);
             }
-        }
 
-        return resolvedNodes;
+            int res = metaStorageMgr.invoke(If.iif(retryPreconditions, successCase, failCase)).get().getAsInt();

Review Comment:
   Another synchronous wait. No timeout, no comments, nothing



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(

Review Comment:
   Padding look too big, is this ok?



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(

Review Comment:
   Variable name isn't the best. And why don't you use static import for `iif`?



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(

Review Comment:
   ... or a static import for `ops`



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()
+                        )
+        );
+
+        return metaStorageMgr.invoke(iif).thenApply(unused -> null);
+    }
+
+    /**
+     * Builds a list of cluster nodes based on a list of peers, pending and stable assignments.
+     * A peer will be added to the result list iff peer's address is present in pending or stable assignments.
+     *
+     * @param peers List of peers.
+     * @param pendingAssignments Byte array that contains serialized list of pending assignments.
+     * @param stableAssignments Byte array that contains serialized list of stable assignments.
+     * @return Resolved cluster nodes.
+     */
+    public static List<ClusterNode> resolveClusterNodes(List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
+        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+
+        if (pendingAssignments != null) {
+            List<ClusterNode> pending = ByteUtils.fromBytes(pendingAssignments);
+            pending.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        if (stableAssignments != null) {
+            List<ClusterNode> stable = ByteUtils.fromBytes(stableAssignments);
+            stable.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+
+        for (PeerId p : peers) {
+            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+
+            if (resolveRegistry.containsKey(addr)) {
+                resolvedNodes.add(resolveRegistry.get(addr));
+            } else {
+                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+            }
+        }
+
+        return resolvedNodes;
+    }
+
+    /**
+     * Reads a list of cluster nodes from a MetaStorage entry.
+     *
+     * @param entry MetaStorage entry.
+     * @return List of cluster nodes.
+     */
+    public static List<ClusterNode> readClusterNodes(Entry entry) {
+        if (entry.empty()) {
+            return Collections.emptyList();
+        }
+
+        return ByteUtils.fromBytes(entry.value());
+    }
+
+    /**
+     * Removes nodes from collection of nodes.
+     *
+     * @param minuend Collection to remove nodes from.
+     * @param subtrahend Collection of nodes to be removed.
+     * @return Result of the subtraction.
+     */
+    public static List<ClusterNode> subtract(Collection<ClusterNode> minuend, Collection<ClusterNode> subtrahend) {
+        return minuend.stream().filter(v -> !subtrahend.contains(v)).collect(Collectors.toList());
+    }
+
+    /**
+     * Adds nodes to the collection of nodes.
+     *
+     * @param op1 First operand.
+     * @param op2 Second operand.
+     * @return Result of the addition.
+     */
+    public static List<ClusterNode> union(Collection<ClusterNode> op1, Collection<ClusterNode> op2) {

Review Comment:
   Why do you have collections as parameters?



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()

Review Comment:
   Hm, what does this do?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);

Review Comment:
   What thread do we use for the handler?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {
+        Integer partitions = tableCfg.partitions().value();
+        Integer replicas = tableCfg.replicas().value();
+        return AffinityUtils.calculateAssignments(baselineMgr.nodes(), partitions, replicas).get(partNum);

Review Comment:
   Oof, this smells so badly. If we only need assignments for a single table, why do we calculate them for potentially hundreds or thousands irrelevant partitions?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {

Review Comment:
   You can move this statement up before "majorityAvailable" is declared, it'll make variable usages more local.



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -423,7 +482,7 @@ private CompletableFuture<?> onTableDelete(ConfigurationNotificationEvent<TableV
                     ctx.storageRevision(),
                     ctx.oldValue().name(),
                     ((ExtendedTableView) ctx.oldValue()).id(),
-                    (List<List<ClusterNode>>) ByteUtils.fromBytes(((ExtendedTableView) ctx.oldValue()).assignments())
+                    ByteUtils.fromBytes(((ExtendedTableView) ctx.oldValue()).assignments())

Review Comment:
   Now we just sweep this bs under the carpet. I believe that embarrassing code like this must be explicit.



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -606,6 +708,32 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
         CompletableFuture.allOf(futures).join();
     }
 
+    /**
+     * Calculates the quantity of the data nodes for the partition of the table.
+     *
+     * @param tblId Table id.
+     * @param partId Partition id.
+     * @param partAssignments Partition assignments.
+     * @return A future that will hold the quantity of data nodes.
+     */
+    private CompletableFuture<Long> queryDataNodesCount(UUID tblId, int partId, List<ClusterNode> partAssignments) {
+        HasDataRequestBuilder requestBuilder = TABLE_MESSAGES_FACTORY.hasDataRequest().tableId(tblId).partitionId(partId);
+
+        //noinspection unchecked
+        CompletableFuture<Boolean>[] requestFutures = partAssignments.stream().map(node -> {
+            HasDataRequest request = requestBuilder.build();
+
+            return raftMgr.messagingService().invoke(node, request, TimeUnit.SECONDS.toMillis(3)).thenApply(response -> {

Review Comment:
   Hard-coded timeout, dude



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {

Review Comment:
   BTW, I'm not validating the logic behind all these computation, otherwise it would take me two more days



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()
+                        )
+        );
+
+        return metaStorageMgr.invoke(iif).thenApply(unused -> null);
+    }
+
+    /**
+     * Builds a list of cluster nodes based on a list of peers, pending and stable assignments.
+     * A peer will be added to the result list iff peer's address is present in pending or stable assignments.
+     *
+     * @param peers List of peers.
+     * @param pendingAssignments Byte array that contains serialized list of pending assignments.
+     * @param stableAssignments Byte array that contains serialized list of stable assignments.
+     * @return Resolved cluster nodes.
+     */
+    public static List<ClusterNode> resolveClusterNodes(List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
+        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+
+        if (pendingAssignments != null) {
+            List<ClusterNode> pending = ByteUtils.fromBytes(pendingAssignments);
+            pending.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        if (stableAssignments != null) {
+            List<ClusterNode> stable = ByteUtils.fromBytes(stableAssignments);
+            stable.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+
+        for (PeerId p : peers) {
+            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+
+            if (resolveRegistry.containsKey(addr)) {
+                resolvedNodes.add(resolveRegistry.get(addr));
+            } else {
+                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+            }
+        }
+
+        return resolvedNodes;
+    }
+
+    /**
+     * Reads a list of cluster nodes from a MetaStorage entry.
+     *
+     * @param entry MetaStorage entry.
+     * @return List of cluster nodes.
+     */
+    public static List<ClusterNode> readClusterNodes(Entry entry) {
+        if (entry.empty()) {
+            return Collections.emptyList();
+        }
+
+        return ByteUtils.fromBytes(entry.value());
+    }
+
+    /**
+     * Removes nodes from collection of nodes.
+     *
+     * @param minuend Collection to remove nodes from.
+     * @param subtrahend Collection of nodes to be removed.
+     * @return Result of the subtraction.
+     */
+    public static List<ClusterNode> subtract(Collection<ClusterNode> minuend, Collection<ClusterNode> subtrahend) {
+        return minuend.stream().filter(v -> !subtrahend.contains(v)).collect(Collectors.toList());
+    }
+
+    /**
+     * Adds nodes to the collection of nodes.
+     *
+     * @param op1 First operand.
+     * @param op2 Second operand.
+     * @return Result of the addition.
+     */
+    public static List<ClusterNode> union(Collection<ClusterNode> op1, Collection<ClusterNode> op2) {

Review Comment:
   How slow can this be btw? How big are these collections?
   I know that "ususally" contains is invoked on a smaller collection. Should we apply such optimization here?



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_REDUCE_PREFIX + partId);
+    }
+
+    /**
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchAppendKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_APPEND_PREFIX + partId);
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
+     *
+     * @param key Key.
+     * @return Table id.
+     */
+    public static UUID extractTableId(ByteArray key) {
+        return extractTableId(key, "");
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
      *
      * @param key Key.
+     * @param prefix Key prefix.
      * @return Table id.
      */
     public static UUID extractTableId(ByteArray key, String prefix) {
-        var strKey = key.toString();
+        String strKey = key.toString();

Review Comment:
   No encoding here



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_REDUCE_PREFIX + partId);
+    }
+
+    /**
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchAppendKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_APPEND_PREFIX + partId);
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
+     *
+     * @param key Key.
+     * @return Table id.
+     */
+    public static UUID extractTableId(ByteArray key) {
+        return extractTableId(key, "");
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
      *
      * @param key Key.
+     * @param prefix Key prefix.
      * @return Table id.
      */
     public static UUID extractTableId(ByteArray key, String prefix) {
-        var strKey = key.toString();
+        String strKey = key.toString();
 
         return UUID.fromString(strKey.substring(prefix.length(), strKey.indexOf("_part_")));

Review Comment:
   Again, it has nothing to do with your code, but don't UUIDs have a fixed length? What the hell is going on with these classes



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {
+        Integer partitions = tableCfg.partitions().value();

Review Comment:
   Quick node, it's better when you use "tableCfg.value()" and then access its properties. It's faster and it doesn't involve boxing.



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()
+                        )
+        );
+
+        return metaStorageMgr.invoke(iif).thenApply(unused -> null);
+    }
+
+    /**
+     * Builds a list of cluster nodes based on a list of peers, pending and stable assignments.
+     * A peer will be added to the result list iff peer's address is present in pending or stable assignments.
+     *
+     * @param peers List of peers.
+     * @param pendingAssignments Byte array that contains serialized list of pending assignments.
+     * @param stableAssignments Byte array that contains serialized list of stable assignments.
+     * @return Resolved cluster nodes.
+     */
+    public static List<ClusterNode> resolveClusterNodes(List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
+        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+
+        if (pendingAssignments != null) {
+            List<ClusterNode> pending = ByteUtils.fromBytes(pendingAssignments);
+            pending.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        if (stableAssignments != null) {
+            List<ClusterNode> stable = ByteUtils.fromBytes(stableAssignments);
+            stable.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+
+        for (PeerId p : peers) {
+            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+
+            if (resolveRegistry.containsKey(addr)) {
+                resolvedNodes.add(resolveRegistry.get(addr));
+            } else {
+                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+            }
+        }
+
+        return resolvedNodes;
+    }
+
+    /**
+     * Reads a list of cluster nodes from a MetaStorage entry.
+     *
+     * @param entry MetaStorage entry.
+     * @return List of cluster nodes.
+     */
+    public static List<ClusterNode> readClusterNodes(Entry entry) {
+        if (entry.empty()) {
+            return Collections.emptyList();
+        }
+
+        return ByteUtils.fromBytes(entry.value());
+    }
+
+    /**
+     * Removes nodes from collection of nodes.
+     *
+     * @param minuend Collection to remove nodes from.
+     * @param subtrahend Collection of nodes to be removed.
+     * @return Result of the subtraction.
+     */
+    public static List<ClusterNode> subtract(Collection<ClusterNode> minuend, Collection<ClusterNode> subtrahend) {

Review Comment:
   Had to google it. These are real words, damn!



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()
+                        )
+        );
+
+        return metaStorageMgr.invoke(iif).thenApply(unused -> null);
+    }
+
+    /**
+     * Builds a list of cluster nodes based on a list of peers, pending and stable assignments.
+     * A peer will be added to the result list iff peer's address is present in pending or stable assignments.
+     *
+     * @param peers List of peers.
+     * @param pendingAssignments Byte array that contains serialized list of pending assignments.
+     * @param stableAssignments Byte array that contains serialized list of stable assignments.
+     * @return Resolved cluster nodes.
+     */
+    public static List<ClusterNode> resolveClusterNodes(List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
+        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+
+        if (pendingAssignments != null) {
+            List<ClusterNode> pending = ByteUtils.fromBytes(pendingAssignments);
+            pending.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        if (stableAssignments != null) {
+            List<ClusterNode> stable = ByteUtils.fromBytes(stableAssignments);
+            stable.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+
+        for (PeerId p : peers) {
+            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+
+            if (resolveRegistry.containsKey(addr)) {
+                resolvedNodes.add(resolveRegistry.get(addr));
+            } else {
+                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+            }
+        }
+
+        return resolvedNodes;
+    }
+
+    /**
+     * Reads a list of cluster nodes from a MetaStorage entry.
+     *
+     * @param entry MetaStorage entry.
+     * @return List of cluster nodes.
+     */
+    public static List<ClusterNode> readClusterNodes(Entry entry) {
+        if (entry.empty()) {
+            return Collections.emptyList();
+        }
+
+        return ByteUtils.fromBytes(entry.value());
+    }
+
+    /**
+     * Removes nodes from collection of nodes.
+     *
+     * @param minuend Collection to remove nodes from.
+     * @param subtrahend Collection of nodes to be removed.
+     * @return Result of the subtraction.
+     */
+    public static List<ClusterNode> subtract(Collection<ClusterNode> minuend, Collection<ClusterNode> subtrahend) {
+        return minuend.stream().filter(v -> !subtrahend.contains(v)).collect(Collectors.toList());
+    }
+
+    /**
+     * Adds nodes to the collection of nodes.
+     *
+     * @param op1 First operand.
+     * @param op2 Second operand.
+     * @return Result of the addition.
+     */
+    public static List<ClusterNode> union(Collection<ClusterNode> op1, Collection<ClusterNode> op2) {
+        return op2.stream().filter(v -> !op1.contains(v)).collect(Collectors.toCollection(() -> new ArrayList<>(op1)));
+    }
+
+    /**
+     * Returns an intersection of two collections of nodes.
+     *
+     * @param op1 First operand.
+     * @param op2 Second operand.
+     * @return Result of the intersection.
+     */
+    public static List<ClusterNode> intersect(Collection<ClusterNode> op1, Collection<ClusterNode> op2) {

Review Comment:
   Same here. Keep in mind that `contains` on lists is slow. That's one of reasons why I'm asking about parameter types. They may hide performance issues.



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {

Review Comment:
   Usually it's `partId` rather then `partNum`, or maybe I'm wrong, I don't know



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {

Review Comment:
   Last reminder about "partId", I hope you'll rename it everywhere



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);

Review Comment:
   Every usage of ByteUtils is a red flag for me. That's just a reminder for the future, it's very slow and produces a lot of boilerplate data. Assignments in Ignite 2.x are stored much more efficiently.



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {

Review Comment:
   What's 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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957446581


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()

Review Comment:
   Just nothing



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957380076


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1562,6 +1697,40 @@ public void onError(@NotNull Throwable e) {
                 LOG.warn("Unable to process stable assignments event", e);
             }
         });
+
+        metaStorageMgr.registerWatchByPrefix(ByteArray.fromString(ASSIGNMENTS_SWITCH_REDUCE_PREFIX), new WatchListener() {
+            @Override
+            public boolean onUpdate(@NotNull WatchEvent evt) {
+                ByteArray key = evt.entryEvent().newEntry().key();
+
+                int partitionNumber = extractPartitionNumber(key);
+                UUID tblId = extractTableId(key, ASSIGNMENTS_SWITCH_REDUCE_PREFIX);
+                String partitionId = partitionRaftGroupName(tblId, partitionNumber);
+
+                TableImpl tbl = tablesByIdVv.latest().get(tblId);
+                ExtendedTableConfiguration tblCfg = (ExtendedTableConfiguration) tablesCfg.tables().get(tbl.name());

Review Comment:
   You're right. I usually relies on IDEA to tell me that cast is redundant :( 



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957378578


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {

Review Comment:
    Well in this code partId is the id of the partition raft group (its name) and part num is the number of the partition, I think we can refactor it later



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957435041


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(

Review Comment:
   oops



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957428590


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {

Review Comment:
   It's just the comment explaining the logic of the metastorage invoke



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958232755


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);

Review Comment:
   It MAY be, but it's used all over, so I think we can leave it for the `Future#get` refactoring ticket that I've created



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958252991


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_REDUCE_PREFIX + partId);
+    }
+
+    /**
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchAppendKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_APPEND_PREFIX + partId);
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
+     *
+     * @param key Key.
+     * @return Table id.
+     */
+    public static UUID extractTableId(ByteArray key) {
+        return extractTableId(key, "");
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
      *
      * @param key Key.
+     * @param prefix Key prefix.
      * @return Table id.
      */
     public static UUID extractTableId(ByteArray key, String prefix) {
-        var strKey = key.toString();
+        String strKey = key.toString();
 
         return UUID.fromString(strKey.substring(prefix.length(), strKey.indexOf("_part_")));

Review Comment:
   Remember when metastorage was in the design phase and people were thinking "we can store keys effectively, like storing integers as 4 bytes instead of their decimal representation...". Everyone forgot about it. But I digress.
   Storing tableId in keys is absolutely fine
   



-- 
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] sashapolo commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r956008392


##########
modules/core/src/main/java/org/apache/ignite/internal/util/ByteUtils.java:
##########
@@ -123,12 +123,12 @@ public static byte[] toBytes(Object obj) {
      * @param bytes Byte array.
      * @return Object.
      */
-    public static Object fromBytes(byte[] bytes) {
+    public static <T> T fromBytes(byte[] bytes) {
         try (
                 var bis = new ByteArrayInputStream(bytes);
                 var in = new ObjectInputStream(bis)
         ) {
-            return in.readObject();
+            return (T) in.readObject();

Review Comment:
   ok, I did a little bit of thinking and this code looks safe and is more convenient to use. Please ignore my 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] sashapolo commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r955959657


##########
modules/core/src/main/java/org/apache/ignite/internal/util/ByteUtils.java:
##########
@@ -123,12 +123,12 @@ public static byte[] toBytes(Object obj) {
      * @param bytes Byte array.
      * @return Object.
      */
-    public static Object fromBytes(byte[] bytes) {
+    public static <T> T fromBytes(byte[] bytes) {
         try (
                 var bis = new ByteArrayInputStream(bytes);
                 var in = new ObjectInputStream(bis)
         ) {
-            return in.readObject();
+            return (T) in.readObject();

Review Comment:
   They are not exactly the same, because with the explicit cast the ClassCastException may be thrown earlier



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

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

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


[GitHub] [ignite-3] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r955922081


##########
modules/network/src/main/java/org/apache/ignite/internal/network/netty/NettyUtils.java:
##########
@@ -38,18 +41,21 @@ public class NettyUtils {
      * @return CompletableFuture.
      */
     public static <T, R, F extends Future<R>> CompletableFuture<T> toCompletableFuture(
-            F nettyFuture,
+            @Schedule F nettyFuture,
             Function<F, T> mapper
     ) {
         var fut = new CompletableFuture<T>();
 
-        nettyFuture.addListener((F future) -> {
-            if (future.isSuccess()) {
-                fut.complete(mapper.apply(future));
-            } else if (future.isCancelled()) {
-                fut.cancel(true);
-            } else {
-                fut.completeExceptionally(future.cause());
+        nettyFuture.addListener(new GenericFutureListener<F>() {
+            @Override
+            public void operationComplete(@Execute F future) throws Exception {

Review Comment:
   TIL that it is possible



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

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

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


[GitHub] [ignite-3] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957351193


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);
+
+                    if (table != null) {
+                        MvTableStorage storage = table.internalTable().storage();
+
+                        MvPartitionStorage mvPartition = storage.getMvPartition(partitionId);
+
+                        if (mvPartition != null) {
+                            // If applied index of a storage is greater than 0,
+                            // then there is data
+                            contains = mvPartition.lastAppliedIndex() > 0;

Review Comment:
   It works as intended because we can't wait for recovery as recovery sends requests like 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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957413658


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -157,6 +171,12 @@ public class RebalanceUtil {
     /** Key prefix for stable assignments. */
     public static final String STABLE_ASSIGNMENTS_PREFIX = "assignments.stable.";
 
+    /** Key prefix for switch reduce assignments. */
+    public static final String ASSIGNMENTS_SWITCH_REDUCE_PREFIX = "assignments.switch.reduce.";

Review Comment:
   It's possible to change values to something like "a.s.r", wdyt about 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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957378578


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -801,6 +929,12 @@ private void dropTableLocally(long causalityToken, String name, UUID tblId, List
         }
     }
 
+    private List<ClusterNode> calculateAssignments(TableConfiguration tableCfg, int partNum) {

Review Comment:
   Well in this code partId is the id of the partition raft group (its name)



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958109473


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
+
+                                        if (majorityAvailable) {
+                                            String partitionId = partitionRaftGroupName(tblId, partId);
+                                            RebalanceUtil.startPeerRemoval(partitionId, localMember, metaStorageMgr);
+                                            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.";
+                                            throw new IgniteInternalException(msg);
+                                        }
+                                    });
+                                } else {
+                                    fut = CompletableFuture.completedFuture(true);
                                 }
+
+                                return fut.thenComposeAsync(startGroup -> {

Review Comment:
   Can we use "composeAsync" or something?
   Is there a way to add assertions that we are not in the network thread? Would be handy I think



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -606,6 +708,32 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
         CompletableFuture.allOf(futures).join();
     }
 
+    /**
+     * Calculates the quantity of the data nodes for the partition of the table.
+     *
+     * @param tblId Table id.
+     * @param partId Partition id.
+     * @param partAssignments Partition assignments.
+     * @return A future that will hold the quantity of data nodes.
+     */
+    private CompletableFuture<Long> queryDataNodesCount(UUID tblId, int partId, List<ClusterNode> partAssignments) {

Review Comment:
   Ok, never mind then



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958107920


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);
+
+                    if (table != null) {
+                        MvTableStorage storage = table.internalTable().storage();
+
+                        MvPartitionStorage mvPartition = storage.getMvPartition(partitionId);
+
+                        if (mvPartition != null) {
+                            // If applied index of a storage is greater than 0,
+                            // then there is data
+                            contains = mvPartition.lastAppliedIndex() > 0;

Review Comment:
   Is it possible to falsely return "false" here while `lastAppliedIndex` is still 0 and it won't be in a moment?



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957341460


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {

Review Comment:
   We only have listeners for groups ATM



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957446581


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()

Review Comment:
   Just nothing, it's like
   
   if (....) {
      // this is the ops(....).yield()
   }
   // absence of else is ops().yield()



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()

Review Comment:
   Just nothing, it's like
   
   ```
   if (....) {
      // this is the ops(....).yield()
   }
   // absence of else is ops().yield()
   ```



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958115746


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_REDUCE_PREFIX + partId);
+    }
+
+    /**
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchAppendKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_APPEND_PREFIX + partId);
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
+     *
+     * @param key Key.
+     * @return Table id.
+     */
+    public static UUID extractTableId(ByteArray key) {
+        return extractTableId(key, "");
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
      *
      * @param key Key.
+     * @param prefix Key prefix.
      * @return Table id.
      */
     public static UUID extractTableId(ByteArray key, String prefix) {
-        var strKey = key.toString();
+        String strKey = key.toString();
 
         return UUID.fromString(strKey.substring(prefix.length(), strKey.indexOf("_part_")));

Review Comment:
   I mean that calling `indexOf` makes no sense! It can be derived from `prefix.length()`. Author clearly didn't have time to think about performance



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957301197


##########
modules/network-annotation-processor/src/main/java/org/apache/ignite/internal/network/processor/messages/MessageImplGenerator.java:
##########
@@ -105,6 +107,7 @@ public TypeSpec generateMessageImpl(MessageClass message, TypeSpec builderInterf
             String getterName = getter.getSimpleName().toString();
 
             FieldSpec.Builder fieldBuilder = FieldSpec.builder(getterReturnType, getterName)
+                    .addAnnotation(IgniteToStringInclude.class)

Review Comment:
   Let's add a ticket for this issue



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957403283


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/RebalanceRaftGroupEventsListener.java:
##########
@@ -252,92 +294,189 @@ private void scheduleChangePeers(List<PeerId> peers, long term) {
      */
     private void doOnNewPeersConfigurationApplied(List<PeerId> peers) {
         try {
-            Map<ByteArray, Entry> keys = metaStorageMgr.getAll(
+            ByteArray pendingPartAssignmentsKey = pendingPartAssignmentsKey(partId);
+            ByteArray stablePartAssignmentsKey = stablePartAssignmentsKey(partId);
+            ByteArray plannedPartAssignmentsKey = plannedPartAssignmentsKey(partId);
+            ByteArray switchReduceKey = switchReduceKey(partId);
+            ByteArray switchAppendKey = switchAppendKey(partId);
+
+            Map<ByteArray, Entry> values = metaStorageMgr.getAll(
                     Set.of(
-                            plannedPartAssignmentsKey(partId),
-                            pendingPartAssignmentsKey(partId),
-                            stablePartAssignmentsKey(partId))).get();
+                            plannedPartAssignmentsKey,
+                            pendingPartAssignmentsKey,
+                            stablePartAssignmentsKey,
+                            switchReduceKey,
+                            switchAppendKey
+                    )
+            ).get();

Review Comment:
   You're right



-- 
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] SammyVimes merged pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes merged PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016


-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r956003401


##########
modules/core/src/main/java/org/apache/ignite/internal/util/ByteUtils.java:
##########
@@ -123,12 +123,12 @@ public static byte[] toBytes(Object obj) {
      * @param bytes Byte array.
      * @return Object.
      */
-    public static Object fromBytes(byte[] bytes) {
+    public static <T> T fromBytes(byte[] bytes) {
         try (
                 var bis = new ByteArrayInputStream(bytes);
                 var in = new ObjectInputStream(bis)
         ) {
-            return in.readObject();
+            return (T) in.readObject();

Review Comment:
   I'm against such code, but for other reasons.
   Technically, both approaches are absolutely the same here. This method will return Object without cast, because no type information is available to it. It's all just sugar.



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958230989


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()

Review Comment:
   I think it's a good idea, yes



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958234310


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {

Review Comment:
   Sure



-- 
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] sk0x50 commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
sk0x50 commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958643102


##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java:
##########
@@ -603,7 +604,17 @@ else if (resp0.errorCode() == RaftError.EBUSY.getNumber() ||
                         resp0.errorCode() == (RaftError.EAGAIN.getNumber()) ||
                         resp0.errorCode() == (RaftError.ENOENT.getNumber())) { // Possibly a node has not been started.
                         executor.schedule(() -> {
-                            sendWithRetry(peer, req, stopTime, fut);
+                            Peer targetPeer = peer;
+
+                            if (resp0.errorCode() == RaftError.ENOENT.getNumber()) {

Review Comment:
   I still insist that separate issues should go under separate tickets regardless their size.



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957339495


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java:
##########
@@ -0,0 +1,354 @@
+/*
+ * 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.runner.app;
+
+import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
+import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * These tests check in-memory node restart scenarios.
+ */
+public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
+    /** Default node port. */
+    private static final int DEFAULT_NODE_PORT = 3344;
+
+    /** Value producer for table data, is used to create data and check it later. */
+    private static final IntFunction<String> VALUE_PRODUCER = i -> "val " + i;
+
+    /** Prefix for full table name. */
+    private static final String SCHEMA_PREFIX = "PUBLIC.";
+
+    /** Test table name. */
+    private static final String TABLE_NAME = "Table1";
+
+    /** Nodes bootstrap configuration pattern. */
+    private static final String NODE_BOOTSTRAP_CFG = "{\n"
+            + "  network.port: {},\n"
+            + "  network.nodeFinder.netClusterNodes: {}\n"
+            + "}";
+
+    /** Cluster nodes. */
+    private static final List<Ignite> CLUSTER_NODES = new ArrayList<>();
+
+    private static final List<String> CLUSTER_NODES_NAMES = new ArrayList<>();
+
+    /**
+     * Stops all started nodes.
+     */
+    @AfterEach
+    public void afterEach() throws Exception {
+        var closeables = new ArrayList<AutoCloseable>();
+
+        for (String name : CLUSTER_NODES_NAMES) {
+            if (name != null) {
+                closeables.add(() -> IgnitionManager.stop(name));
+            }
+        }
+
+        CLUSTER_NODES.clear();
+        CLUSTER_NODES_NAMES.clear();
+
+        IgniteUtils.closeAll(closeables);
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @param nodeName Node name.
+     * @param cfgString Configuration string.
+     * @param workDir Working directory.
+     * @return Created node instance.
+     */
+    private static IgniteImpl startNode(int idx, String nodeName, @Nullable String cfgString, Path workDir) {
+        assertTrue(CLUSTER_NODES.size() == idx || CLUSTER_NODES.get(idx) == null);
+
+        CLUSTER_NODES_NAMES.add(idx, nodeName);
+
+        CompletableFuture<Ignite> future = IgnitionManager.start(nodeName, cfgString, workDir.resolve(nodeName));
+
+        if (CLUSTER_NODES.isEmpty()) {
+            IgnitionManager.init(nodeName, List.of(nodeName), "cluster");
+        }
+
+        assertThat(future, willCompleteSuccessfully());
+
+        Ignite ignite = future.join();
+
+        CLUSTER_NODES.add(idx, ignite);
+
+        return (IgniteImpl) ignite;
+    }
+
+    /**
+     * Start node with the given parameters.
+     *
+     * @param testInfo Test info.
+     * @param idx Node index, is used to stop the node later, see {@link #stopNode(int)}.
+     * @return Created node instance.
+     */
+    private IgniteImpl startNode(TestInfo testInfo, int idx) {
+        int port = DEFAULT_NODE_PORT + idx;
+        String nodeName = testNodeName(testInfo, port);
+        String cfgString = configurationString(idx, null, null);
+
+        return startNode(idx, nodeName, cfgString, workDir.resolve(nodeName));
+    }
+
+    /**
+     * Build a configuration string.
+     *
+     * @param idx Node index.
+     * @param cfg Optional configuration string.
+     * @param predefinedPort Predefined port.
+     * @return Configuration string.
+     */
+    private static String configurationString(int idx, @Nullable String cfg, @Nullable Integer predefinedPort) {
+        int port = predefinedPort == null ? DEFAULT_NODE_PORT + idx : predefinedPort;
+        int connectPort = predefinedPort == null ? DEFAULT_NODE_PORT : predefinedPort;
+        String connectAddr = "[\"localhost:" + connectPort + "\"]";
+
+        return cfg == null ? IgniteStringFormatter.format(NODE_BOOTSTRAP_CFG, port, connectAddr) : cfg;
+    }
+
+    /**
+     * Stop the node with given index.
+     *
+     * @param idx Node index.
+     */
+    private static void stopNode(int idx) {

Review Comment:
   No reason, it's actually a copy-paste from a different test



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

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

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


[GitHub] [ignite-3] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957353533


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -365,6 +380,50 @@ public CompletableFuture<Boolean> notify(@NotNull SchemaEventParameters paramete
                 return completedFuture(false);
             }
         });
+
+        addMessageHandler(raftMgr.messagingService());
+    }
+
+    /**
+     * Adds a table manager message handler.
+     *
+     * @param messagingService Messaging service.
+     */
+    private void addMessageHandler(MessagingService messagingService) {
+        var messageHandler = new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (message instanceof HasDataRequest) {
+                    // This message queries if a node has any data for a specific partition of a table
+                    assert correlationId != null;
+
+                    HasDataRequest msg = (HasDataRequest) message;
+
+                    UUID tableId = msg.tableId();
+                    int partitionId = msg.partitionId();
+
+                    boolean contains = false;
+
+                    TableImpl table = tablesByIdVv.latest().get(tableId);
+
+                    if (table != null) {
+                        MvTableStorage storage = table.internalTable().storage();
+
+                        MvPartitionStorage mvPartition = storage.getMvPartition(partitionId);
+
+                        if (mvPartition != null) {
+                            // If applied index of a storage is greater than 0,
+                            // then there is data
+                            contains = mvPartition.lastAppliedIndex() > 0;

Review Comment:
   Added a 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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958250361


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
+
+                                        if (majorityAvailable) {
+                                            String partitionId = partitionRaftGroupName(tblId, partId);
+                                            RebalanceUtil.startPeerRemoval(partitionId, localMember, metaStorageMgr);
+                                            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.";
+                                            throw new IgniteInternalException(msg);
+                                        }
+                                    });
+                                } else {
+                                    fut = CompletableFuture.completedFuture(true);
                                 }
+
+                                return fut.thenComposeAsync(startGroup -> {

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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957445993


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(

Review Comment:
   If we could do metastorage operations as code, this would be just an `if`. I don't know how else I should call it. Any other name won't grasp the meaning of it.



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(

Review Comment:
   Right



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957271853


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);
+        List<ClusterNode> switchReduce = (List<ClusterNode>) ByteUtils.fromBytes(eventData);
+
+        ByteArray pendingKey = pendingPartAssignmentsKey(partId);
+
+        List<ClusterNode> pendingAssignments = subtract(assignments, switchReduce);
+
+        byte[] pendingByteArray = ByteUtils.toBytes(pendingAssignments);
+        byte[] assignmentsByteArray = ByteUtils.toBytes(assignments);
+
+        if (switchReduce.isEmpty()) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        ByteArray changeTriggerKey = partChangeTriggerKey(partId);
+        byte[] rev = ByteUtils.longToBytes(event.entryEvent().newEntry().revision());
+
+        // if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey) && notExists(stableKey)) {
+        //     put(pendingKey, pending)
+        //     put(stableKey, assignments)
+        //     put(changeTriggerKey, revision)
+        // } else if ((notExists(changeTriggerKey) || value(changeTriggerKey) < revision) && (notExists(pendingKey))) {
+        //     put(pendingKey, pending)
+        //     put(changeTriggerKey, revision)
+        // }
+
+        If iif = If.iif(
+                        and(
+                                or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                and(notExists(pendingKey), (notExists(stablePartAssignmentsKey(partId))))
+                        ),
+                        Operations.ops(
+                                put(pendingKey, pendingByteArray),
+                                put(stablePartAssignmentsKey(partId), assignmentsByteArray),
+                                put(changeTriggerKey, rev)
+                        ).yield(),
+                        If.iif(
+                            and(
+                                    or(notExists(changeTriggerKey), value(changeTriggerKey).lt(rev)),
+                                    notExists(pendingKey)
+                            ),
+                            Operations.ops(
+                                    put(pendingKey, pendingByteArray),
+                                    put(changeTriggerKey, rev)
+                            ).yield(),
+                            ops().yield()
+                        )
+        );
+
+        return metaStorageMgr.invoke(iif).thenApply(unused -> null);
+    }
+
+    /**
+     * Builds a list of cluster nodes based on a list of peers, pending and stable assignments.
+     * A peer will be added to the result list iff peer's address is present in pending or stable assignments.
+     *
+     * @param peers List of peers.
+     * @param pendingAssignments Byte array that contains serialized list of pending assignments.
+     * @param stableAssignments Byte array that contains serialized list of stable assignments.
+     * @return Resolved cluster nodes.
+     */
+    public static List<ClusterNode> resolveClusterNodes(List<PeerId> peers, byte[] pendingAssignments, byte[] stableAssignments) {
+        Map<NetworkAddress, ClusterNode> resolveRegistry = new HashMap<>();
+
+        if (pendingAssignments != null) {
+            List<ClusterNode> pending = ByteUtils.fromBytes(pendingAssignments);
+            pending.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        if (stableAssignments != null) {
+            List<ClusterNode> stable = ByteUtils.fromBytes(stableAssignments);
+            stable.forEach(n -> resolveRegistry.put(n.address(), n));
+        }
+
+        List<ClusterNode> resolvedNodes = new ArrayList<>(peers.size());
+
+        for (PeerId p : peers) {
+            var addr = NetworkAddress.from(p.getEndpoint().getIp() + ":" + p.getEndpoint().getPort());
+
+            if (resolveRegistry.containsKey(addr)) {
+                resolvedNodes.add(resolveRegistry.get(addr));
+            } else {
+                throw new IgniteInternalException("Can't find appropriate cluster node for raft group peer: " + p);
+            }
+        }
+
+        return resolvedNodes;
+    }
+
+    /**
+     * Reads a list of cluster nodes from a MetaStorage entry.
+     *
+     * @param entry MetaStorage entry.
+     * @return List of cluster nodes.
+     */
+    public static List<ClusterNode> readClusterNodes(Entry entry) {
+        if (entry.empty()) {
+            return Collections.emptyList();
+        }
+
+        return ByteUtils.fromBytes(entry.value());
+    }
+
+    /**
+     * Removes nodes from collection of nodes.
+     *
+     * @param minuend Collection to remove nodes from.
+     * @param subtrahend Collection of nodes to be removed.
+     * @return Result of the subtraction.
+     */
+    public static List<ClusterNode> subtract(Collection<ClusterNode> minuend, Collection<ClusterNode> subtrahend) {
+        return minuend.stream().filter(v -> !subtrahend.contains(v)).collect(Collectors.toList());
+    }
+
+    /**
+     * Adds nodes to the collection of nodes.
+     *
+     * @param op1 First operand.
+     * @param op2 Second operand.
+     * @return Result of the addition.
+     */
+    public static List<ClusterNode> union(Collection<ClusterNode> op1, Collection<ClusterNode> op2) {

Review Comment:
   How slow can this be btw? How big are these collections?
   I know that "usually" contains is invoked on a smaller collection. Should we apply such optimization 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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957426017


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -235,4 +288,209 @@ public static boolean recoverable(Throwable t) {
         // As long as we don't have a general failure handler, we assume that all errors are recoverable.
         return true;
     }
+
+    /**
+     * Starts the process of removing peer from raft group if that peer has in-memory storage or if its
+     * storage was cleared.
+     *
+     * @param partId Partition's raft group id.
+     * @param clusterNode Cluster node to be removed from peers.
+     * @param metaStorageMgr MetaStorage manager.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> startPeerRemoval(String partId, ClusterNode clusterNode, MetaStorageManager metaStorageMgr) {
+        ByteArray key = switchReduceKey(partId);
+
+        return metaStorageMgr.get(key)
+                .thenCompose(retrievedAssignmentsSwitchReduce -> {
+                    byte[] prevValue = retrievedAssignmentsSwitchReduce.value();
+
+                    boolean prevValueEmpty = true;
+                    List<ClusterNode> calculatedAssignmentsSwitchReduce;
+
+                    if (prevValue != null) {
+                        calculatedAssignmentsSwitchReduce = (List<ClusterNode>) ByteUtils.fromBytes(prevValue);
+                        prevValueEmpty = false;
+                    } else {
+                        calculatedAssignmentsSwitchReduce = new ArrayList<>();
+                    }
+
+                    calculatedAssignmentsSwitchReduce.add(clusterNode);
+
+                    byte[] newValue = ByteUtils.toBytes(calculatedAssignmentsSwitchReduce);
+
+                    if (prevValueEmpty) {
+                        return metaStorageMgr.invoke(
+                                Conditions.notExists(key),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    } else {
+                        return metaStorageMgr.invoke(
+                                revision(key).eq(retrievedAssignmentsSwitchReduce.revision()),
+                                Operations.put(key, newValue),
+                                Operations.noop()
+                        );
+                    }
+                }).thenCompose(res -> {
+                    if (!res) {
+                        return startPeerRemoval(partId, clusterNode, metaStorageMgr);
+                    }
+
+                    return CompletableFuture.completedFuture(null);
+                });
+    }
+
+    /**
+     * Handles assignments switch reduce changed.
+     *
+     * @param metaStorageMgr MetaStorage manager.
+     * @param baselineNodes Baseline nodes.
+     * @param partitions Partitions count.
+     * @param replicas Replicas count.
+     * @param partNum Number of the partition.
+     * @param partId Partition's raft group id..
+     * @param event Assignments switch reduce change event.
+     * @return Completable future that signifies the completion of this operation.
+     */
+    public static CompletableFuture<Void> handleReduceChanged(MetaStorageManager metaStorageMgr, Collection<ClusterNode> baselineNodes,
+            int partitions, int replicas, int partNum, String partId, WatchEvent event) {
+        Entry entry = event.entryEvent().newEntry();
+        byte[] eventData = entry.value();
+
+        List<ClusterNode> assignments = AffinityUtils.calculateAssignments(baselineNodes, partitions, replicas).get(partNum);

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] sk0x50 commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
sk0x50 commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958401991


##########
modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeMessageTypes.java:
##########
@@ -24,7 +24,7 @@
 /**
  * Message types for the Compute module.
  */
-@MessageGroup(groupName = "ComputeMessages", groupType = 6)
+@MessageGroup(groupType = 6, groupName = "ComputeMessages")

Review Comment:
   What is the purpose of this change? :)



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957359315


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
+
+                                        if (majorityAvailable) {
+                                            String partitionId = partitionRaftGroupName(tblId, partId);
+                                            RebalanceUtil.startPeerRemoval(partitionId, localMember, metaStorageMgr);
+                                            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.";
+                                            throw new IgniteInternalException(msg);
+                                        }
+                                    });
+                                } else {
+                                    fut = CompletableFuture.completedFuture(true);
                                 }
+
+                                return fut.thenComposeAsync(startGroup -> {

Review Comment:
   Well in this case fut.thenCompose will be running in network thread, because `queryDataNodesCount` completes on the network thread



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957355850


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -423,7 +482,7 @@ private CompletableFuture<?> onTableDelete(ConfigurationNotificationEvent<TableV
                     ctx.storageRevision(),
                     ctx.oldValue().name(),
                     ((ExtendedTableView) ctx.oldValue()).id(),
-                    (List<List<ClusterNode>>) ByteUtils.fromBytes(((ExtendedTableView) ctx.oldValue()).assignments())
+                    ByteUtils.fromBytes(((ExtendedTableView) ctx.oldValue()).assignments())

Review Comment:
   But then it will be both embarassing and eye-bleeding



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957414072


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {

Review Comment:
   The latter one



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

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

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


[GitHub] [ignite-3] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958233820


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -552,33 +618,69 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
 
                 if (raftMgr.shouldHaveRaftGroupLocally(nodes)) {
                     startGroupFut = CompletableFuture
-                            .supplyAsync(
-                                    () -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
+                            .supplyAsync(() -> internalTbl.storage().getOrCreateMvPartition(partId), ioExecutor)
                             .thenComposeAsync((partitionStorage) -> {
-                                RaftGroupOptions groupOptions = groupOptionsForPartition(internalTbl, tblCfg, partitionStorage,
-                                        newPartAssignment);
-
-                                try {
-                                    raftMgr.startRaftGroupNode(
-                                            grpId,
-                                            newPartAssignment,
-                                            new PartitionListener(tblId, new VersionedRowStore(partitionStorage, txManager)),
-                                            new RebalanceRaftGroupEventsListener(
-                                                    metaStorageMgr,
-                                                    tablesCfg.tables().get(tablesById.get(tblId).name()),
-                                                    grpId,
-                                                    partId,
-                                                    busyLock,
-                                                    movePartition(() -> internalTbl.partitionRaftGroupService(partId)),
-                                                    rebalanceScheduler
-                                            ),
-                                            groupOptions
-                                    );
-
-                                    return CompletableFuture.completedFuture(null);
-                                } catch (NodeStoppingException ex) {
-                                    return CompletableFuture.failedFuture(ex);
+                                boolean hasData = partitionStorage.lastAppliedIndex() > 0;
+
+                                CompletableFuture<Boolean> fut;
+
+                                if (isInMemory || !hasData) {
+                                    List<ClusterNode> partAssignments = assignmentsLatest.get(partId);
+
+                                    fut = queryDataNodesCount(tblId, partId, partAssignments).thenApply(dataNodesCount -> {
+                                        boolean fullPartitionRestart = dataNodesCount == 0;
+                                        boolean majorityAvailable = dataNodesCount >= (partAssignments.size() / 2) + 1;
+
+                                        if (fullPartitionRestart) {
+                                            return true;
+                                        }
+
+                                        if (majorityAvailable) {
+                                            String partitionId = partitionRaftGroupName(tblId, partId);
+                                            RebalanceUtil.startPeerRemoval(partitionId, localMember, metaStorageMgr);
+                                            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.";
+                                            throw new IgniteInternalException(msg);
+                                        }
+                                    });
+                                } else {
+                                    fut = CompletableFuture.completedFuture(true);
                                 }
+
+                                return fut.thenComposeAsync(startGroup -> {

Review Comment:
   There is a way, but I think we should implement a general solution for this problem and not adding a single assertion 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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957299833


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java:
##########
@@ -529,10 +532,12 @@ private final class Watcher extends Thread {
             /**
              * Constructor.
              *
+             * @param nodeId Node id.
              * @param cursor Watch event cursor.
              * @param lsnr   The listener which receives and handles watch updates.
              */
-            Watcher(Cursor<WatchEvent> cursor, WatchListener lsnr) {
+            Watcher(String nodeId, Cursor<WatchEvent> cursor, WatchListener lsnr) {
+                setName(nodeId);

Review Comment:
   How about `ms-watcher-${nodeName}`?



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958102498


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java:
##########
@@ -529,10 +532,12 @@ private final class Watcher extends Thread {
             /**
              * Constructor.
              *
+             * @param nodeId Node id.
              * @param cursor Watch event cursor.
              * @param lsnr   The listener which receives and handles watch updates.
              */
-            Watcher(Cursor<WatchEvent> cursor, WatchListener lsnr) {
+            Watcher(String nodeId, Cursor<WatchEvent> cursor, WatchListener lsnr) {
+                setName(nodeId);

Review Comment:
   Looks good to me



-- 
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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958227665


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java:
##########
@@ -202,13 +222,46 @@ public static ByteArray stablePartAssignmentsKey(String partId) {
     }
 
     /**
-     * Extract table id from pending key of partition.
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchReduceKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_REDUCE_PREFIX + partId);
+    }
+
+    /**
+     * Key that is needed for the rebalance algorithm.
+     *
+     * @param partId Unique identifier of a partition.
+     * @return Key for a partition.
+     * @see <a href="https://github.com/apache/ignite-3/blob/main/modules/table/tech-notes/rebalance.md">Rebalnce documentation</a>
+     */
+    public static ByteArray switchAppendKey(String partId) {
+        return new ByteArray(ASSIGNMENTS_SWITCH_APPEND_PREFIX + partId);
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
+     *
+     * @param key Key.
+     * @return Table id.
+     */
+    public static UUID extractTableId(ByteArray key) {
+        return extractTableId(key, "");
+    }
+
+    /**
+     * Extract table id from a metastorage key of partition.
      *
      * @param key Key.
+     * @param prefix Key prefix.
      * @return Table id.
      */
     public static UUID extractTableId(ByteArray key, String prefix) {
-        var strKey = key.toString();
+        String strKey = key.toString();
 
         return UUID.fromString(strKey.substring(prefix.length(), strKey.indexOf("_part_")));

Review Comment:
   I don't like this method at all, extracting table id from the key seems odd



-- 
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] ibessonov commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r958249542


##########
modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/SimpleCondition.java:
##########
@@ -77,6 +77,8 @@ public long revision() {
          * @throws IllegalStateException In the case when the condition is already defined.
          */
         public SimpleCondition eq(long rev) {
+            assert rev > 0 : "Revision must be positive.";

Review Comment:
   I mean, you can write your own? Anyways, let's leave assertions, I don't really mind 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] SammyVimes commented on a diff in pull request #1016: IGNITE-17196 In-memory partition rebalance

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on code in PR #1016:
URL: https://github.com/apache/ignite-3/pull/1016#discussion_r957401885


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1562,6 +1697,40 @@ public void onError(@NotNull Throwable e) {
                 LOG.warn("Unable to process stable assignments event", e);
             }
         });
+
+        metaStorageMgr.registerWatchByPrefix(ByteArray.fromString(ASSIGNMENTS_SWITCH_REDUCE_PREFIX), new WatchListener() {
+            @Override
+            public boolean onUpdate(@NotNull WatchEvent evt) {
+                ByteArray key = evt.entryEvent().newEntry().key();
+
+                int partitionNumber = extractPartitionNumber(key);
+                UUID tblId = extractTableId(key, ASSIGNMENTS_SWITCH_REDUCE_PREFIX);
+                String partitionId = partitionRaftGroupName(tblId, partitionNumber);
+
+                TableImpl tbl = tablesByIdVv.latest().get(tblId);
+                ExtendedTableConfiguration tblCfg = (ExtendedTableConfiguration) tablesCfg.tables().get(tbl.name());

Review Comment:
   We need to calculate the latest target assignments to proceed with rebalance, I don't think something can go wrong 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