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


##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java:
##########
@@ -773,21 +774,21 @@ private CompletableFuture<CmgRaftService> 
startCmgRaftService(Set<String> nodeNa
         assert serverPeer != null;
 
         try {
-            return raftManager
-                    .startRaftGroupNodeAndWaitNodeReadyFuture(
-                            raftNodeId(serverPeer),
-                            raftConfiguration,
-                            new CmgRaftGroupListener(
-                                    clusterStateStorageMgr,
-                                    logicalTopology,
-                                    validationManager,
-                                    this::onLogicalTopologyChanged,
-                                    clusterIdStore
-                            ),
-                            this::onElectedAsLeader,
-                            raftGroupOptionsConfigurer
-                    )
-                    .thenApply(service -> new CmgRaftService(service, 
clusterService, logicalTopology));
+            RaftGroupService service = 
raftManager.startRaftGroupNodeAndWaitNodeReady(
+                    raftNodeId(serverPeer),
+                    raftConfiguration,
+                    new CmgRaftGroupListener(
+                            clusterStateStorageMgr,
+                            logicalTopology,
+                            validationManager,
+                            this::onLogicalTopologyChanged,
+                            clusterIdStore
+                    ),
+                    this::onElectedAsLeader,
+                    raftGroupOptionsConfigurer
+            );
+
+            return completedFuture(new CmgRaftService(service, clusterService, 
logicalTopology));

Review Comment:
   Seems that there's no need in CompletableFuture in `private 
CompletableFuture<CmgRaftService> startCmgRaftService` any longer. Please 
create a follow-up ticket for an issues like that.



##########
modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLearnersTest.java:
##########
@@ -228,40 +229,34 @@ void testAddLearners() {
 
         PeersAndLearners configuration = 
createConfiguration(List.of(follower), List.of());
 
-        CompletableFuture<RaftGroupService> service1 = startRaftGroup(
-                follower,
-                configuration.peer(follower.consistentId()),
-                configuration,
-                new TestRaftGroupListener()
-        );
+        RaftGroupService service1 =
+                startRaftGroup(follower, 
configuration.peer(follower.consistentId()), configuration, new 
TestRaftGroupListener());

Review Comment:
   One param at a line I guess.



##########
modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java:
##########
@@ -498,13 +498,11 @@ private RaftGroupService startClient(TestInfo testInfo, 
TestReplicationGroupId g
 
         Marshaller commandsMarshaller = commandsMarshaller(clientNode);
 
-        CompletableFuture<RaftGroupService> clientFuture = RaftGroupServiceImpl
-                .start(groupId, clientNode, FACTORY, raftConfiguration, 
initialMemberConf, true, executor, commandsMarshaller);
+        RaftGroupService clientFuture = RaftGroupServiceImpl

Review Comment:
   "clientFuture" - It's no longer future.



##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java:
##########
@@ -1027,20 +1031,21 @@ private <T> CompletableFuture<T> 
doWithOneOffRaftGroupService(
             PeersAndLearners raftClientConfiguration,
             Function<RaftGroupService, CompletableFuture<T>> action
     ) {
-        return startOneOffRaftGroupService(raftClientConfiguration)
-                .thenCompose(raftGroupService -> action.apply(raftGroupService)
-                        .whenComplete((res, ex) -> raftGroupService.shutdown())
-                );
-    }
-
-    private CompletableFuture<RaftGroupService> 
startOneOffRaftGroupService(PeersAndLearners newConfiguration) {
         try {
-            return raftMgr.startRaftGroupService(MetastorageGroupId.INSTANCE, 
newConfiguration);
+            RaftGroupService raftGroupService = 
startOneOffRaftGroupService(raftClientConfiguration);
+
+            return action.apply(raftGroupService)
+                    .whenComplete((res, ex) -> raftGroupService.shutdown());
         } catch (NodeStoppingException e) {
             return failedFuture(e);
         }
     }
 
+    private RaftGroupService startOneOffRaftGroupService(PeersAndLearners 
newConfiguration)

Review Comment:
   The helper method has only one usage and is very simple inside. I'd rather 
use `raftMgr.startRaftGroupService(MetastorageGroupId.INSTANCE, 
newConfiguration);` instead of helper.



##########
modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLearnersTest.java:
##########
@@ -409,19 +387,17 @@ void testChangePeersToAddLearnerToSameNodeAsPeer() throws 
InterruptedException {
 
         PeersAndLearners newConfiguration = createConfiguration(followers, 
List.of(learner, newLearner));
 
-        CompletableFuture<Void> changePeersFuture = 
learnerService.thenCompose(s -> s.refreshAndGetLeaderWithTerm()
-                .thenCompose(leaderWithTerm -> 
s.changePeersAndLearnersAsync(newConfiguration, leaderWithTerm.term())
-        ));
+        CompletableFuture<Void> changePeersFuture = 
learnerService.refreshAndGetLeaderWithTerm()
+                .thenCompose(leaderWithTerm -> 
learnerService.changePeersAndLearnersAsync(newConfiguration, 
leaderWithTerm.term()));
 
         assertThat(changePeersFuture, willCompleteSuccessfully());
 
         var newLearnerListener = new TestRaftGroupListener();
 
-        CompletableFuture<RaftGroupService> newLearnerService = startRaftGroup(
+        startRaftGroup(
                 newLearner, 
newConfiguration.learner(newLearner.consistentId()), newConfiguration, 
newLearnerListener

Review Comment:
   Same as above. One at a line. Up to you though.



##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java:
##########
@@ -462,28 +464,30 @@ private CompletableFuture<? extends RaftGroupService> 
startRaftNode(
                 }
         );
 
-        serviceFuture
-                .thenAccept(service -> service.subscribeLeader(new 
MetaStorageLeaderElectionListener(
-                        busyLock,
-                        clusterService,
-                        logicalTopologyService,
-                        metaStorageSvcFut,
-                        learnerManager,
-                        clusterTime,
-                        // We use the "deployWatchesFuture" to guarantee that 
the Configuration Manager will be started
-                        // when the underlying code tries to read Meta Storage 
configuration. This is a consequence of having a circular
-                        // dependency between these two components.
-                        deployWatchesFuture.thenApply(v -> 
localMetaStorageConfiguration),
-                        electionListeners,
-                        this::peersChangeStateExists
-                )))
-                .whenComplete((v, e) -> {
+        LeaderElectionListener leaderElectionListener = new 
MetaStorageLeaderElectionListener(
+                busyLock,
+                clusterService,
+                logicalTopologyService,
+                metaStorageSvcFut,
+                learnerManager,
+                clusterTime,
+                // We use the "deployWatchesFuture" to guarantee that the 
Configuration Manager will be started
+                // when the underlying code tries to read Meta Storage 
configuration. This is a consequence of having a circular
+                // dependency between these two components.
+                deployWatchesFuture.thenApply(v -> 
localMetaStorageConfiguration),
+                electionListeners,
+                this::peersChangeStateExists
+        );
+
+        return completedFuture(service)

Review Comment:
   What about 
   ```
           return service.subscribeLeader(leaderElectionListener)
                   .handle((v, e) -> {
                       if (e != null) {
                           LOG.error("Unable to register 
MetaStorageLeaderElectionListener", e);
                       }
   
                       return service;
                   });
   ```
   ?



-- 
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-unsubscr...@ignite.apache.org

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

Reply via email to