sanpwc commented on code in PR #6641:
URL: https://github.com/apache/ignite-3/pull/6641#discussion_r2386987314


##########
modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/AssignmentsTracker.java:
##########
@@ -156,6 +184,124 @@ public CompletableFuture<List<TokenizedAssignments>> 
getAssignments(
                 }));
     }
 
+    @Override
+    public CompletableFuture<List<TokenizedAssignments>> 
awaitNonEmptyAssignments(
+            List<? extends ReplicationGroupId> replicationGroupIds,
+            HybridTimestamp clusterTimeToAwait,
+            long timeoutMillis
+    ) {
+        return msManager
+                .clusterTime()
+                .waitFor(clusterTimeToAwait)
+                .thenCompose(ignored -> inBusyLock(busyLock, () -> {
+                    long now = coarseCurrentTimeMillis();
+                    return 
awaitNonEmptyAssignmentsWithCheckMostRecent(replicationGroupIds, now, 
timeoutMillis);
+                }))
+                .thenApply(identity());
+    }
+
+    private CompletableFuture<List<TokenizedAssignments>> 
awaitNonEmptyAssignmentsWithCheckMostRecent(
+            List<? extends ReplicationGroupId> replicationGroupIds,
+            long startTime,
+            long timeoutMillis
+    ) {
+        Map<ReplicationGroupId, TokenizedAssignments> assignmentsMap = 
stableAssignments();
+
+        Map<Integer, CompletableFuture<TokenizedAssignments>> futures = new 
HashMap<>();
+        List<TokenizedAssignments> result = new 
ArrayList<>(replicationGroupIds.size());
+
+        for (int i = 0; i < replicationGroupIds.size(); i++) {
+            ReplicationGroupId groupId = replicationGroupIds.get(i);
+
+            TokenizedAssignments a = assignmentsMap.get(groupId);
+            result.add(a);
+
+            if (a.nodes().isEmpty()) {
+                if (timeoutMillis > 0) {
+                    futures.put(i, nonEmptyAssignmentFuture(groupId, 
timeoutMillis));
+                } else {
+                    // If timeout is zero or less, then this group is failed, 
the correct exception will be thrown
+                    // in #checkEmptyAssignmentsReasons().
+                    futures.put(i, failedFuture(new TimeoutException()));
+                }
+            }
+        }
+
+        if (futures.isEmpty()) {
+            return completedFuture(result);
+        } else {
+            return allOf(futures.values())
+                    .handle((unused, ex) -> {
+                        if (ex == null) {
+                            // Get the most recent assignments after the 
waiting.
+                            long now = System.currentTimeMillis();
+                            long newTimeoutMillis = timeoutMillis - (now - 
startTime);
+                            return 
awaitNonEmptyAssignmentsWithCheckMostRecent(replicationGroupIds, startTime, 
newTimeoutMillis);

Review Comment:
   Why not to simply retrieve corresponding assignments from 
groupStableAssignments() instead of calling 
awaitNonEmptyAssignmentsWithCheckMostRecent()? I mean that at this point 
because of allOf(futures.values()) all futures should be already completed.



##########
modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/AssignmentsTracker.java:
##########
@@ -156,6 +184,124 @@ public CompletableFuture<List<TokenizedAssignments>> 
getAssignments(
                 }));
     }
 
+    @Override
+    public CompletableFuture<List<TokenizedAssignments>> 
awaitNonEmptyAssignments(
+            List<? extends ReplicationGroupId> replicationGroupIds,
+            HybridTimestamp clusterTimeToAwait,
+            long timeoutMillis
+    ) {
+        return msManager
+                .clusterTime()
+                .waitFor(clusterTimeToAwait)
+                .thenCompose(ignored -> inBusyLock(busyLock, () -> {
+                    long now = coarseCurrentTimeMillis();
+                    return 
awaitNonEmptyAssignmentsWithCheckMostRecent(replicationGroupIds, now, 
timeoutMillis);
+                }))
+                .thenApply(identity());
+    }
+
+    private CompletableFuture<List<TokenizedAssignments>> 
awaitNonEmptyAssignmentsWithCheckMostRecent(
+            List<? extends ReplicationGroupId> replicationGroupIds,
+            long startTime,
+            long timeoutMillis
+    ) {
+        Map<ReplicationGroupId, TokenizedAssignments> assignmentsMap = 
stableAssignments();
+
+        Map<Integer, CompletableFuture<TokenizedAssignments>> futures = new 
HashMap<>();
+        List<TokenizedAssignments> result = new 
ArrayList<>(replicationGroupIds.size());
+
+        for (int i = 0; i < replicationGroupIds.size(); i++) {
+            ReplicationGroupId groupId = replicationGroupIds.get(i);
+
+            TokenizedAssignments a = assignmentsMap.get(groupId);
+            result.add(a);
+
+            if (a.nodes().isEmpty()) {

Review Comment:
   Is it possible for a to be null?



##########
modules/runner/src/testFixtures/java/org/apache/ignite/internal/AssignmentsTestUtils.java:
##########
@@ -75,34 +75,4 @@ public static void awaitAssignmentsStabilization(Ignite 
node, String tableName)
             return totalPartitionSize == zone.partitions() * zone.replicas();
         }, 10_000));
     }
-
-    /**
-     * Returns a future that completes when the Default Zone's primary 
replicas have been elected.
-     */
-    // TODO: remove this method after 
https://issues.apache.org/jira/browse/IGNITE-25283 has been fixed.
-    public static void awaitAssignmentsStabilizationOnDefaultZone(Ignite node) 
throws InterruptedException {

Review Comment:
   awaitAssignmentsStabilization() is also expected to be removed.



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/mapping/ExecutionDistributionProviderImpl.java:
##########
@@ -46,6 +46,9 @@
 
 /** Execution nodes information provider. */
 public class ExecutionDistributionProviderImpl implements 
ExecutionDistributionProvider {
+    /** Non-empty assignments await timeout. */
+    public static final int AWAIT_NON_EMPTY_ASSIGNMENTS_TIMEOUT_MILLIS = 
30_000;

Review Comment:
   Why it's 30 seconds? Is allReplicas called within a transaction? If true it 
should be limited to tx timeout.



##########
modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/AssignmentsTracker.java:
##########
@@ -156,6 +184,124 @@ public CompletableFuture<List<TokenizedAssignments>> 
getAssignments(
                 }));
     }
 
+    @Override
+    public CompletableFuture<List<TokenizedAssignments>> 
awaitNonEmptyAssignments(
+            List<? extends ReplicationGroupId> replicationGroupIds,
+            HybridTimestamp clusterTimeToAwait,
+            long timeoutMillis
+    ) {
+        return msManager
+                .clusterTime()
+                .waitFor(clusterTimeToAwait)
+                .thenCompose(ignored -> inBusyLock(busyLock, () -> {
+                    long now = coarseCurrentTimeMillis();
+                    return 
awaitNonEmptyAssignmentsWithCheckMostRecent(replicationGroupIds, now, 
timeoutMillis);
+                }))
+                .thenApply(identity());
+    }
+
+    private CompletableFuture<List<TokenizedAssignments>> 
awaitNonEmptyAssignmentsWithCheckMostRecent(
+            List<? extends ReplicationGroupId> replicationGroupIds,
+            long startTime,
+            long timeoutMillis
+    ) {
+        Map<ReplicationGroupId, TokenizedAssignments> assignmentsMap = 
stableAssignments();
+
+        Map<Integer, CompletableFuture<TokenizedAssignments>> futures = new 
HashMap<>();
+        List<TokenizedAssignments> result = new 
ArrayList<>(replicationGroupIds.size());
+
+        for (int i = 0; i < replicationGroupIds.size(); i++) {
+            ReplicationGroupId groupId = replicationGroupIds.get(i);
+
+            TokenizedAssignments a = assignmentsMap.get(groupId);
+            result.add(a);
+
+            if (a.nodes().isEmpty()) {
+                if (timeoutMillis > 0) {
+                    futures.put(i, nonEmptyAssignmentFuture(groupId, 
timeoutMillis));
+                } else {
+                    // If timeout is zero or less, then this group is failed, 
the correct exception will be thrown
+                    // in #checkEmptyAssignmentsReasons().
+                    futures.put(i, failedFuture(new TimeoutException()));
+                }
+            }
+        }
+
+        if (futures.isEmpty()) {
+            return completedFuture(result);
+        } else {
+            return allOf(futures.values())
+                    .handle((unused, ex) -> {
+                        if (ex == null) {
+                            // Get the most recent assignments after the 
waiting.
+                            long now = System.currentTimeMillis();
+                            long newTimeoutMillis = timeoutMillis - (now - 
startTime);
+                            return 
awaitNonEmptyAssignmentsWithCheckMostRecent(replicationGroupIds, startTime, 
newTimeoutMillis);
+                        } else {
+                            return 
checkEmptyAssignmentsReasons(replicationGroupIds, futures, ex);
+                        }
+                    })
+                    .thenCompose(identity());
+        }
+    }
+
+    private CompletableFuture<List<TokenizedAssignments>> 
checkEmptyAssignmentsReasons(

Review Comment:
   I guess it's a bit excessive. allOf() will be completed exceptionally on the 
very first exceptionally completed future, thus you may just unwrap the cause 
instead of iterating over all futures in assignmentFuturesMap.



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

To unsubscribe, e-mail: [email protected]

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

Reply via email to