You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tk...@apache.org on 2022/09/26 13:37:42 UTC

[ignite-3] branch main updated: IGNITE-17493 Add 'isLeader' check to CMG topology listener (#1120)

This is an automated email from the ASF dual-hosted git repository.

tkalkirill pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 01ccc036f7 IGNITE-17493 Add 'isLeader' check to CMG topology listener (#1120)
01ccc036f7 is described below

commit 01ccc036f7139be6fd750062af14c24dcaf9c457
Author: Alexander Polovtcev <al...@gmail.com>
AuthorDate: Mon Sep 26 16:37:37 2022 +0300

    IGNITE-17493 Add 'isLeader' check to CMG topology listener (#1120)
---
 .../management/ClusterManagementGroupManager.java  | 30 ++++++++++++----------
 .../cluster/management/raft/CmgRaftService.java    |  1 -
 2 files changed, 17 insertions(+), 14 deletions(-)

diff --git a/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java b/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
index 54cb6bbc88..3db35e4bfe 100644
--- a/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
+++ b/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
@@ -457,7 +457,7 @@ public class ClusterManagementGroupManager implements IgniteComponent {
                                         + "Re-creating the CMG Raft service [reason={}]", e, e.getMessage());
                             } else {
                                 LOG.debug("CMG service started, but the cluster state is different. "
-                                        + "Re-creating the CMG Raft service [localState={}, clusterState={}]",
+                                                + "Re-creating the CMG Raft service [localState={}, clusterState={}]",
                                         service.nodeNames(), state.cmgNodes());
 
                                 destroyCmg();
@@ -526,17 +526,10 @@ public class ClusterManagementGroupManager implements IgniteComponent {
         return new TopologyEventHandler() {
             @Override
             public void onAppeared(ClusterNode member) {
-                raftService.readClusterState()
-                        .thenAccept(state -> {
-                            if (state != null) {
-                                sendClusterState(state, member)
-                                        .whenComplete((v, e) -> {
-                                            if (e != null) {
-                                                LOG.info("Unable to send cluster state", e);
-                                            }
-                                        });
-                            } else {
-                                LOG.info("Unable to send cluster state to a newly added node. Cluster state is empty [node={}]", member);
+                raftService.isCurrentNodeLeader()
+                        .thenAccept(isLeader -> {
+                            if (isLeader) {
+                                sendClusterState(raftService, member);
                             }
                         });
             }
@@ -559,6 +552,17 @@ public class ClusterManagementGroupManager implements IgniteComponent {
         }, 0, TimeUnit.MILLISECONDS);
     }
 
+    private void sendClusterState(CmgRaftService raftService, ClusterNode node) {
+        raftService.readClusterState()
+                .thenAccept(state -> {
+                    if (state != null) {
+                        sendClusterState(state, node);
+                    } else {
+                        LOG.warn("Unable to send cluster state to a newly added node. Cluster state is empty [node={}]", node);
+                    }
+                });
+    }
+
     private CompletableFuture<Void> sendClusterState(ClusterState clusterState, ClusterNode node) {
         NetworkMessage msg = msgFactory.clusterStateMessage()
                 .clusterState(clusterState)
@@ -586,7 +590,7 @@ public class ClusterManagementGroupManager implements IgniteComponent {
 
         return result.whenComplete((v, e) -> {
             if (e != null) {
-                LOG.info("Unable to send message [msg={}, target={}]", e, msg.getClass(), node);
+                LOG.warn("Unable to send message [msg={}, target={}]", e, msg.getClass(), node);
             }
         });
     }
diff --git a/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftService.java b/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftService.java
index b1580c0ae9..17feaa1c0c 100644
--- a/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftService.java
+++ b/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftService.java
@@ -70,7 +70,6 @@ public class CmgRaftService {
      *
      * @return {@code true} if the current node is the CMG leader.
      */
-    // TODO: replace with onLeaderElected callback https://issues.apache.org/jira/browse/IGNITE-17493
     public CompletableFuture<Boolean> isCurrentNodeLeader() {
         ClusterNode thisNode = clusterService.topologyService().localMember();