Author: brandonwilliams Date: Mon Feb 14 19:55:30 2011 New Revision: 1070628
URL: http://svn.apache.org/viewvc?rev=1070628&view=rev Log: Keep endpoint state until aVeryLongTime when not a fat client Patch by brandonwilliams, reviewed by gdusbabek for CASSANDRA-2115 Modified: cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java Modified: cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java?rev=1070628&r1=1070627&r2=1070628&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java Mon Feb 14 19:55:30 2011 @@ -253,17 +253,18 @@ public class Gossiper implements IFailur } /** - * Removes the endpoint from unreachable endpoint set + * Removes the endpoint from gossip completely * * @param endpoint endpoint to be removed from the current membership. */ private void evictFromMembership(InetAddress endpoint) { unreachableEndpoints.remove(endpoint); + endpointStateMap.remove(endpoint); } /** - * Removes the endpoint completely from Gossip + * Removes the endpoint from Gossip but retains endpoint state */ public void removeEndpoint(InetAddress endpoint) { @@ -445,20 +446,15 @@ public class Gossiper implements IFailur { long duration = now - epState.getUpdateTimestamp(); + if (StorageService.instance.getTokenMetadata().isMember(endpoint)) + epState.setHasToken(true); // check if this is a fat client. fat clients are removed automatically from // gosip after FatClientTimeout - if (!epState.getHasToken() && !epState.isAlive() && (duration > FatClientTimeout)) + if (!epState.getHasToken() && !epState.isAlive() && !justRemovedEndpoints.containsKey(endpoint) && (duration > FatClientTimeout)) { - if (StorageService.instance.getTokenMetadata().isMember(endpoint)) - epState.setHasToken(true); - else - { - if (!justRemovedEndpoints.containsKey(endpoint)) // if the node was decommissioned, it will have been removed but still appear as a fat client - { - logger.info("FatClient " + endpoint + " has been silent for " + FatClientTimeout + "ms, removing from gossip"); - removeEndpoint(endpoint); // after quarantine justRemoveEndpoints will remove the state - } - } + logger.info("FatClient " + endpoint + " has been silent for " + FatClientTimeout + "ms, removing from gossip"); + removeEndpoint(endpoint); // will put it in justRemovedEndpoints to respect quarantine delay + evictFromMembership(endpoint); // can get rid of the state immediately } if ( !epState.isAlive() && (duration > aVeryLongTime) ) @@ -477,7 +473,6 @@ public class Gossiper implements IFailur if (logger.isDebugEnabled()) logger.debug(QUARANTINE_DELAY + " elapsed, " + entry.getKey() + " gossip quarantine over"); justRemovedEndpoints.remove(entry.getKey()); - endpointStateMap.remove(entry.getKey()); } } }
