JAkutenshi commented on code in PR #7813:
URL: https://github.com/apache/ignite-3/pull/7813#discussion_r3014805077


##########
modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java:
##########
@@ -784,6 +785,12 @@ public class IgniteImpl implements Ignite {
                 raftGroupEventsClientListener
         );
 
+        var msRaftServiceFactory = new 
PhysicalTopologyAwareRaftGroupServiceFactory(

Review Comment:
   ```suggestion
           var msRaftGroupServiceFactory = new 
PhysicalTopologyAwareRaftGroupServiceFactory(
   ```



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/client/PhysicalTopologyAwareRaftGroupService.java:
##########
@@ -327,6 +327,13 @@ public void unsubscribeLeader(LeaderElectionListener 
callback) {
 
     @Override
     public <R> CompletableFuture<R> run(Command cmd, long timeoutMillis) {
+        if (commandExecutor.leader() == null && timeoutMillis != 0) {
+            // Discover the leader first, similar to 
RaftGroupServiceImpl.run() behavior.
+            // Without this, the command is sent to a random peer causing 
cascading EPERM errors
+            // and potential hangs due to race conditions with leader election 
notifications.
+            return refreshAndGetLeaderWithTerm(timeoutMillis)
+                    .thenCompose(ignored -> commandExecutor.run(cmd, 
timeoutMillis));
+        }

Review Comment:
   ```suggestion
           }
           
   ```



##########
modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java:
##########
@@ -277,16 +281,21 @@ public void 
testRangeNextWorksCorrectlyAfterLeaderChange() throws Exception {
                 .orElseThrow()
                 .value;
 
+        // Wrap RaftGroupService in a TimeAwareRaftGroupService mock for 
MetaStorageServiceImpl.
+        TimeAwareRaftGroupService timeAwareService = 
mock(TimeAwareRaftGroupService.class);

Review Comment:
   I'm suspicious with this mock because of integration test. Should it be more 
proper to start timeout aware services in `prepareJraftMetaStorages()`?



##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java:
##########
@@ -504,7 +505,7 @@ private MetaStorageServiceImpl 
createMetaStorageService(RaftGroupService raftSer
         );
     }
 
-    private CompletableFuture<? extends RaftGroupService> startVotingNode(
+    private CompletableFuture<PhysicalTopologyAwareRaftGroupService> 
startVotingNode(

Review Comment:
   Should we specify implementation type instead if the just interface there?



##########
modules/raft/src/test/java/org/apache/ignite/internal/raft/client/PhysicalTopologyAwareRaftGroupServiceRunTest.java:
##########
@@ -334,43 +355,31 @@ void testZeroTimeoutTriesAllPeersBeforeFailing() {
     }
 
     /**
-     * Tests that with Long.MAX_VALUE timeout, all peers are tried first, then 
waits for leader, and succeeds when leader appears.
+     * Tests that with Long.MAX_VALUE timeout, run() discovers the leader via 
GetLeaderRequest
+     * and then succeeds with the actual command.
+     *
+     * <p>With the new behavior, run() calls refreshAndGetLeaderWithTerm() 
first when leader is unknown.
+     * The GetLeaderRequest retries until leader is discovered, then the 
WriteActionRequest is sent.
      */
     @Test
     void testInfiniteTimeoutWaitsForLeaderAndSucceeds() throws Exception {
-        // First 3 WriteActionRequest calls return EPERM (no leader), then 
success after leader appears.
-        AtomicInteger callCount = new AtomicInteger(0);
-        CountDownLatch allPeersTried = new CountDownLatch(3);
-
-        when(messagingService.invoke(
-                any(InternalClusterNode.class),
-                argThat(this::isTestWriteCommand),
-                anyLong()
-        )).thenAnswer(invocation -> {
-            if (callCount.incrementAndGet() <= 3) {
-                allPeersTried.countDown();
-                return completedFuture(FACTORY.errorResponse()
-                        .errorCode(RaftError.EPERM.getNumber())
-                        .build());
-            }
-            return completedFuture(FACTORY.actionResponse().result(new 
TestResponse()).build());
-        });
+        // WriteActionRequest always succeeds (sent after leader is 
discovered).
+        mockUserInputSuccess();

Review Comment:
   I faced issues with repeated runs. This one, 
`testLeaderWaitModeRetriesNoLeaderPeersAfterLeaderElection` and the one 
`testGetLeaderRequestTriesDifferentPeerOnTransientError([3] ENOENT)` that is 
flaky on TC now. Can I ask to recheck 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: [email protected]

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

Reply via email to