Updated Branches: refs/heads/trunk 44a7db706 -> db59808bf
Remove EndpointState.hasToken Patch by brandonwilliams, reviewed by vijay for CASSANDRA-3747 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/db59808b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/db59808b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/db59808b Branch: refs/heads/trunk Commit: db59808bf73c7c4709ea5b33b5ba1d59d7582b69 Parents: 44a7db7 Author: Brandon Williams <[email protected]> Authored: Mon Jan 16 16:31:32 2012 -0600 Committer: Brandon Williams <[email protected]> Committed: Mon Jan 16 16:31:32 2012 -0600 ---------------------------------------------------------------------- .../org/apache/cassandra/gms/EndpointState.java | 17 -------------- src/java/org/apache/cassandra/gms/Gossiper.java | 18 +++++---------- 2 files changed, 6 insertions(+), 29 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/db59808b/src/java/org/apache/cassandra/gms/EndpointState.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/gms/EndpointState.java b/src/java/org/apache/cassandra/gms/EndpointState.java index c6783a7..0e643cd 100644 --- a/src/java/org/apache/cassandra/gms/EndpointState.java +++ b/src/java/org/apache/cassandra/gms/EndpointState.java @@ -46,12 +46,6 @@ public class EndpointState private volatile long updateTimestamp; private volatile boolean isAlive; - // whether this endpoint has token associated with it or not. Initially set false for all - // endpoints. After certain time of inactivity, gossiper will examine if this node has a - // token or not and will set this true if token is found. If there is no token, this is a - // fat client and will be removed automatically from gossip. - private volatile boolean hasToken; - public static IVersionedSerializer<EndpointState> serializer() { return serializer; @@ -62,7 +56,6 @@ public class EndpointState hbState = initialHbState; updateTimestamp = System.currentTimeMillis(); isAlive = true; - hasToken = false; } HeartBeatState getHeartBeatState() @@ -120,16 +113,6 @@ public class EndpointState { isAlive = false; } - - void setHasToken(boolean value) - { - hasToken = value; - } - - boolean hasToken() - { - return hasToken; - } } class EndpointStateSerializer implements IVersionedSerializer<EndpointState> http://git-wip-us.apache.org/repos/asf/cassandra/blob/db59808b/src/java/org/apache/cassandra/gms/Gossiper.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index d1a5b8d..27a1ba7 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -622,19 +622,15 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean long duration = now - epState.getUpdateTimestamp(); // check if this is a fat client. fat clients are removed automatically from - // gosip after FatClientTimeout - if (!epState.hasToken() && !epState.isAlive() && !justRemovedEndpoints.containsKey(endpoint) && (duration > FatClientTimeout)) + // gossip after FatClientTimeout. Do not remove dead states here. + if (!isDeadState(epState) && !epState.isAlive() && !StorageService.instance.getTokenMetadata().isMember(endpoint) && !justRemovedEndpoints.containsKey(endpoint) && (duration > FatClientTimeout)) { - if (StorageService.instance.getTokenMetadata().isMember(endpoint)) - epState.setHasToken(true); - else - { - 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 - } + 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 } + // check for dead state removal long expireTime = getExpireTimeForEndpoint(endpoint); if (!epState.isAlive() && (now > expireTime) && (!StorageService.instance.getTokenMetadata().isMember(endpoint))) @@ -851,7 +847,6 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean { logger.debug("Not marking " + ep + " alive due to dead state"); markDead(ep, epState); - epState.setHasToken(true); // fat clients won't have a dead state } for (IEndpointStateChangeSubscriber subscriber : subscribers) subscriber.onJoin(ep, epState); @@ -1102,7 +1097,6 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean } EndpointState epState = new EndpointState(new HeartBeatState(0)); epState.markDead(); - epState.setHasToken(true); endpointStateMap.put(ep, epState); unreachableEndpoints.put(ep, System.currentTimeMillis()); if (logger.isTraceEnabled())
