tkalkirill commented on code in PR #6019:
URL: https://github.com/apache/ignite-3/pull/6019#discussion_r2139855231


##########
modules/network/src/main/java/org/apache/ignite/internal/network/scalecube/ScaleCubeTopologyService.java:
##########
@@ -88,64 +88,108 @@ void onMembershipEvent(MembershipEvent event) {
         ClusterNode member = fromMember(event.member(), metadata);
 
         if (event.isAdded()) {
-            members.put(member.address(), member);
-            Map<UUID, ClusterNode> clusterNodes = 
membersByConsistentId.computeIfAbsent(member.name(), k -> new 
ConcurrentHashMap<>());
-            clusterNodes.put(member.id(), member);
-            idToMemberMap.put(member.id(), member);
-
-            LOG.info("Node joined [node={}]", member);
-
-            fireAppearedEvent(member);
+            onAddedEvent(member);
         } else if (event.isUpdated()) {
-            members.put(member.address(), member);
-            membersByConsistentId.computeIfAbsent(member.name(), k -> new 
ConcurrentHashMap<>()).put(member.id(), member);
-            membersByConsistentIdInLogicalTopology.compute(member.name(), 
(consId, node) -> {
-                if (node != null && node.id().equals(member.id())) {
-                    return member;
-                }
-                return node;
-            });
-            idToMemberMap.put(member.id(), member);
+            onUpdatedEvent(member);
         } else if (event.isRemoved() || event.isLeaving()) {
-            // We treat LEAVING as 'node left' because the node will not be 
back and we don't want to wait for the suspicion timeout.
+            onRemovedOrLeftEvent(event, member);
+        }
+
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Topology snapshot [nodes={}]", 
members.values().stream().map(ClusterNode::name).collect(Collectors.toList()));
+        }
+    }
+
+    private void onAddedEvent(ClusterNode member) {
+        @Nullable ClusterNode differentNodeWithSameAddress = 
replaceMemberByAddress(member);
+
+        replaceMemberByConsistentId(member, differentNodeWithSameAddress);
+
+        replaceMemberById(member, differentNodeWithSameAddress);
+
+        LOG.info("Node joined [node={}]", member);
+
+        fireAppearedEvent(member);
+    }
+
+    private @Nullable ClusterNode replaceMemberByAddress(ClusterNode member) {
+        ClusterNode prevNodeWithSameAddress = members.put(member.address(), 
member);
+        return prevNodeWithSameAddress == null || 
prevNodeWithSameAddress.id().equals(member.id())
+                ? null : prevNodeWithSameAddress;
+    }
+
+    private void replaceMemberByConsistentId(ClusterNode member, @Nullable 
ClusterNode differentNodeWithSameAddress) {
+        membersByConsistentId.compute(member.name(), (name, 
nodesWithGivenConsistentId) -> {
+            if (nodesWithGivenConsistentId == null) {
+                nodesWithGivenConsistentId = new ConcurrentHashMap<>();
+            }
+
+            if (differentNodeWithSameAddress != null) {
+                
nodesWithGivenConsistentId.remove(differentNodeWithSameAddress.id());
+            }
+            nodesWithGivenConsistentId.put(member.id(), member);
+
+            return nodesWithGivenConsistentId;
+        });
+    }
+
+    private void replaceMemberById(ClusterNode member, @Nullable ClusterNode 
differentNodeWithSameAddress) {
+        idToMemberMap.put(member.id(), member);
+        if (differentNodeWithSameAddress != null) {
+            idToMemberMap.remove(differentNodeWithSameAddress.id());
+        }
+    }
 
-            members.compute(member.address(), (addr, node) -> {
+    private void onUpdatedEvent(ClusterNode member) {
+        @Nullable ClusterNode differentNodeWithSameAddress = 
replaceMemberByAddress(member);
+
+        replaceMemberByConsistentId(member, differentNodeWithSameAddress);
+
+        membersByConsistentIdInLogicalTopology.compute(member.name(), (consId, 
existingNode) -> {
+            if (existingNode != null && existingNode.id().equals(member.id())) 
{
+                return member;
+            }
+            return existingNode;
+        });
+
+        replaceMemberById(member, differentNodeWithSameAddress);
+    }
+
+    private void onRemovedOrLeftEvent(MembershipEvent event, ClusterNode 
member) {
+        // We treat LEAVING as 'node left' because the node will not be back 
and we don't want to wait for the suspicion timeout.
+
+        members.compute(member.address(), (addr, node) -> {
+            if (node == null || node.id().equals(member.id())) {
+                LOG.info("Node left [member={}, eventType={}]", member, 
event.type());
+
+                return null;
+            } else {
                 // Ignore stale remove event.
-                if (node == null || node.id().equals(member.id())) {
-                    LOG.info("Node left [member={}, eventType={}]", member, 
event.type());
+                LOG.info("Node left (noop as it has already reappeared) 
[member={}, eventType={}]", member, event.type());

Review Comment:
   Maybe we should also check here whether info logging is available?



##########
modules/network/src/main/java/org/apache/ignite/internal/network/scalecube/ScaleCubeTopologyService.java:
##########
@@ -88,64 +88,108 @@ void onMembershipEvent(MembershipEvent event) {
         ClusterNode member = fromMember(event.member(), metadata);
 
         if (event.isAdded()) {
-            members.put(member.address(), member);
-            Map<UUID, ClusterNode> clusterNodes = 
membersByConsistentId.computeIfAbsent(member.name(), k -> new 
ConcurrentHashMap<>());
-            clusterNodes.put(member.id(), member);
-            idToMemberMap.put(member.id(), member);
-
-            LOG.info("Node joined [node={}]", member);
-
-            fireAppearedEvent(member);
+            onAddedEvent(member);
         } else if (event.isUpdated()) {
-            members.put(member.address(), member);
-            membersByConsistentId.computeIfAbsent(member.name(), k -> new 
ConcurrentHashMap<>()).put(member.id(), member);
-            membersByConsistentIdInLogicalTopology.compute(member.name(), 
(consId, node) -> {
-                if (node != null && node.id().equals(member.id())) {
-                    return member;
-                }
-                return node;
-            });
-            idToMemberMap.put(member.id(), member);
+            onUpdatedEvent(member);
         } else if (event.isRemoved() || event.isLeaving()) {
-            // We treat LEAVING as 'node left' because the node will not be 
back and we don't want to wait for the suspicion timeout.
+            onRemovedOrLeftEvent(event, member);
+        }
+
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Topology snapshot [nodes={}]", 
members.values().stream().map(ClusterNode::name).collect(Collectors.toList()));
+        }
+    }
+
+    private void onAddedEvent(ClusterNode member) {
+        @Nullable ClusterNode differentNodeWithSameAddress = 
replaceMemberByAddress(member);
+
+        replaceMemberByConsistentId(member, differentNodeWithSameAddress);
+
+        replaceMemberById(member, differentNodeWithSameAddress);
+
+        LOG.info("Node joined [node={}]", member);

Review Comment:
   Maybe we should also check here whether info logging is available?



##########
modules/network/src/main/java/org/apache/ignite/internal/network/scalecube/ScaleCubeTopologyService.java:
##########
@@ -88,64 +88,108 @@ void onMembershipEvent(MembershipEvent event) {
         ClusterNode member = fromMember(event.member(), metadata);
 
         if (event.isAdded()) {
-            members.put(member.address(), member);
-            Map<UUID, ClusterNode> clusterNodes = 
membersByConsistentId.computeIfAbsent(member.name(), k -> new 
ConcurrentHashMap<>());
-            clusterNodes.put(member.id(), member);
-            idToMemberMap.put(member.id(), member);
-
-            LOG.info("Node joined [node={}]", member);
-
-            fireAppearedEvent(member);
+            onAddedEvent(member);
         } else if (event.isUpdated()) {
-            members.put(member.address(), member);
-            membersByConsistentId.computeIfAbsent(member.name(), k -> new 
ConcurrentHashMap<>()).put(member.id(), member);
-            membersByConsistentIdInLogicalTopology.compute(member.name(), 
(consId, node) -> {
-                if (node != null && node.id().equals(member.id())) {
-                    return member;
-                }
-                return node;
-            });
-            idToMemberMap.put(member.id(), member);
+            onUpdatedEvent(member);
         } else if (event.isRemoved() || event.isLeaving()) {
-            // We treat LEAVING as 'node left' because the node will not be 
back and we don't want to wait for the suspicion timeout.
+            onRemovedOrLeftEvent(event, member);
+        }
+
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Topology snapshot [nodes={}]", 
members.values().stream().map(ClusterNode::name).collect(Collectors.toList()));
+        }
+    }
+
+    private void onAddedEvent(ClusterNode member) {
+        @Nullable ClusterNode differentNodeWithSameAddress = 
replaceMemberByAddress(member);
+
+        replaceMemberByConsistentId(member, differentNodeWithSameAddress);
+
+        replaceMemberById(member, differentNodeWithSameAddress);
+
+        LOG.info("Node joined [node={}]", member);
+
+        fireAppearedEvent(member);
+    }
+
+    private @Nullable ClusterNode replaceMemberByAddress(ClusterNode member) {
+        ClusterNode prevNodeWithSameAddress = members.put(member.address(), 
member);
+        return prevNodeWithSameAddress == null || 
prevNodeWithSameAddress.id().equals(member.id())
+                ? null : prevNodeWithSameAddress;
+    }
+
+    private void replaceMemberByConsistentId(ClusterNode member, @Nullable 
ClusterNode differentNodeWithSameAddress) {
+        membersByConsistentId.compute(member.name(), (name, 
nodesWithGivenConsistentId) -> {
+            if (nodesWithGivenConsistentId == null) {
+                nodesWithGivenConsistentId = new ConcurrentHashMap<>();
+            }
+
+            if (differentNodeWithSameAddress != null) {
+                
nodesWithGivenConsistentId.remove(differentNodeWithSameAddress.id());

Review Comment:
   If the map is new, then this operation is not necessary, maybe we can handle 
this situation?



-- 
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