You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2021/09/08 16:55:15 UTC
[cassandra] branch cassandra-3.11 updated: Make assassinate more
resilient to missing tokens
This is an automated email from the ASF dual-hosted git repository.
brandonwilliams pushed a commit to branch cassandra-3.11
in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/cassandra-3.11 by this push:
new 5cdddcf Make assassinate more resilient to missing tokens
5cdddcf is described below
commit 5cdddcf393145b4fb794cc4a73391fad78f58bac
Author: Brandon Williams <br...@apache.org>
AuthorDate: Thu Aug 12 10:48:17 2021 -0500
Make assassinate more resilient to missing tokens
Patch by Robert Stupp and brandonwilliams; reviewed by adelapena for
CASSANDRA-16847
---
CHANGES.txt | 1 +
src/java/org/apache/cassandra/gms/Gossiper.java | 8 +++++---
src/java/org/apache/cassandra/locator/TokenMetadata.java | 2 +-
3 files changed, 7 insertions(+), 4 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index b9668c9..f589208 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
3.11.12
+ * Make assassinate more resilient to missing tokens (CASSANDRA-16847)
* Exclude Jackson 1.x transitive dependency of hadoop* provided dependencies (CASSANDRA-16854)
* Validate SASI tokenizer options before adding index to schema (CASSANDRA-15135)
* Fixup scrub output when no data post-scrub and clear up old use of row, which really means partition (CASSANDRA-16835)
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index 1603693..08cd106 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -658,7 +658,6 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
InetAddress endpoint = InetAddress.getByName(address);
runInGossipStageBlocking(() -> {
EndpointState epState = endpointStateMap.get(endpoint);
- Collection<Token> tokens = null;
logger.warn("Assassinating {} via gossip", endpoint);
if (epState == null)
@@ -683,6 +682,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
epState.getHeartBeatState().forceNewerGenerationUnsafe();
}
+ Collection<Token> tokens = null;
try
{
tokens = StorageService.instance.getTokenMetadata().getTokens(endpoint);
@@ -690,8 +690,10 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
catch (Throwable th)
{
JVMStabilityInspector.inspectThrowable(th);
- // TODO this is broken
- logger.warn("Unable to calculate tokens for {}. Will use a random one", address);
+ }
+ if (tokens == null || tokens.isEmpty())
+ {
+ logger.warn("Trying to assassinate an endpoint {} that does not have any tokens assigned. This should not have happened, trying to continue with a random token.", address);
tokens = Collections.singletonList(StorageService.instance.getTokenMetadata().partitioner.getRandomToken());
}
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index cdd3885..8300a39 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -559,11 +559,11 @@ public class TokenMetadata
public Collection<Token> getTokens(InetAddress endpoint)
{
assert endpoint != null;
- assert isMember(endpoint); // don't want to return nulls
lock.readLock().lock();
try
{
+ assert isMember(endpoint); // don't want to return nulls
return new ArrayList<>(tokenToEndpointMap.inverse().get(endpoint));
}
finally
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org