From 6f2fda5d1f8f89e11a155b4a8bb44aae9f7ad07b Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Thu, 26 Jan 2017 09:43:47 -0600 Subject: [PATCH] Use keyspace primary ranges for CASSANDRA-9639 The primary ranges will differ based on the keyspace and replication strategy/factor. Particularly true of vnode/multi DC environments where the TokenMetadata.getPredecessor of each local token is not necessarily the bounds of the primary ranges. --- src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java index 9c199b66a8ef..29273bde2ff9 100644 --- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java +++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java @@ -68,12 +68,10 @@ public void run() logger.trace("Recording size estimates"); - // find primary token ranges for the local node. - Collection localTokens = StorageService.instance.getLocalTokens(); - Collection> localRanges = metadata.getPrimaryRangesFor(localTokens); - for (Keyspace keyspace : Keyspace.nonLocalStrategy()) { + Collection> localRanges = StorageService.instance.getPrimaryRangesForEndpoint(keyspace.getName(), + FBUtilities.getBroadcastAddress()); for (ColumnFamilyStore table : keyspace.getColumnFamilyStores()) { long start = System.nanoTime();