Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 9D33319F13 for ; Thu, 21 Apr 2016 21:18:56 +0000 (UTC) Received: (qmail 74647 invoked by uid 500); 21 Apr 2016 21:18:56 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 74608 invoked by uid 500); 21 Apr 2016 21:18:56 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 74587 invoked by uid 99); 21 Apr 2016 21:18:56 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 21 Apr 2016 21:18:56 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 42D32DFBC6; Thu, 21 Apr 2016 21:18:56 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: tylerhobbs@apache.org To: commits@cassandra.apache.org Date: Thu, 21 Apr 2016 21:18:56 -0000 Message-Id: <755ebd149e584c2483d17350d1b22667@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/2] cassandra git commit: Filter out all LocalStrat keyspaces for streaming Repository: cassandra Updated Branches: refs/heads/trunk 848352fae -> de1a96c8d Filter out all LocalStrat keyspaces for streaming Patch by Tyler Hobbs; reviewed by Blake Eggleston for CASSANDRA-11627 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/a4e11828 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/a4e11828 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/a4e11828 Branch: refs/heads/trunk Commit: a4e1182816909761c98355b1079b2f9de8efc4bd Parents: d8d920d Author: Tyler Hobbs Authored: Thu Apr 21 16:15:50 2016 -0500 Committer: Tyler Hobbs Committed: Thu Apr 21 16:15:50 2016 -0500 ---------------------------------------------------------------------- CHANGES.txt | 2 ++ .../org/apache/cassandra/config/Schema.java | 13 ++++++++++ src/java/org/apache/cassandra/db/Keyspace.java | 5 ++++ .../cassandra/db/SizeEstimatesRecorder.java | 2 +- .../org/apache/cassandra/dht/BootStrapper.java | 3 +-- .../service/PendingRangeCalculatorService.java | 6 +++-- .../cassandra/service/StorageService.java | 27 +++++++++++--------- .../cassandra/service/StorageServiceMBean.java | 2 ++ .../org/apache/cassandra/tools/NodeProbe.java | 5 ++++ .../org/apache/cassandra/tools/NodeTool.java | 21 ++++++++++++--- .../cassandra/tools/nodetool/Cleanup.java | 2 +- .../apache/cassandra/tools/nodetool/Repair.java | 2 +- .../apache/cassandra/dht/BootStrapperTest.java | 2 +- .../cassandra/locator/SimpleStrategyTest.java | 4 +-- .../service/LeaveAndBootstrapTest.java | 4 +-- .../org/apache/cassandra/service/MoveTest.java | 4 +-- 16 files changed, 75 insertions(+), 29 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index eb2405c..0ec6aef 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 3.0.6 + * Ignore all LocalStrategy keyspaces for streaming and other related + operations (CASSANDRA-11627) * Ensure columnfilter covers indexed columns for thrift 2i queries (CASSANDRA-11523) * Only open one sstable scanner per sstable (CASSANDRA-11412) * Option to specify ProtocolVersion in cassandra-stress (CASSANDRA-11410) http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/config/Schema.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java index 3fd9f11..03d8e8b 100644 --- a/src/java/org/apache/cassandra/config/Schema.java +++ b/src/java/org/apache/cassandra/config/Schema.java @@ -20,6 +20,7 @@ package org.apache.cassandra.config; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.*; +import java.util.stream.Collectors; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -38,6 +39,7 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.UserType; import org.apache.cassandra.index.Index; import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.locator.LocalStrategy; import org.apache.cassandra.repair.SystemDistributedKeyspace; import org.apache.cassandra.schema.*; import org.apache.cassandra.service.MigrationManager; @@ -348,6 +350,17 @@ public class Schema } /** + * @return a collection of keyspaces that do not use LocalStrategy for replication + */ + public List getNonLocalStrategyKeyspaces() + { + return keyspaces.values().stream() + .filter(keyspace -> keyspace.params.replication.klass != LocalStrategy.class) + .map(keyspace -> keyspace.name) + .collect(Collectors.toList()); + } + + /** * @return collection of the user defined keyspaces */ public List getUserKeyspaces() http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/db/Keyspace.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index 2b62f0e..5783b41 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -621,6 +621,11 @@ public class Keyspace return Iterables.transform(Schema.instance.getNonSystemKeyspaces(), keyspaceTransformer); } + public static Iterable nonLocalStrategy() + { + return Iterables.transform(Schema.instance.getNonLocalStrategyKeyspaces(), keyspaceTransformer); + } + public static Iterable system() { return Iterables.transform(Schema.SYSTEM_KEYSPACE_NAMES, keyspaceTransformer); http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java index e838281..2a74ea9 100644 --- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java +++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java @@ -72,7 +72,7 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable Collection localTokens = StorageService.instance.getLocalTokens(); Collection> localRanges = metadata.getPrimaryRangesFor(localTokens); - for (Keyspace keyspace : Keyspace.nonSystem()) + for (Keyspace keyspace : Keyspace.nonLocalStrategy()) { for (ColumnFamilyStore table : keyspace.getColumnFamilyStores()) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/dht/BootStrapper.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java index 8d8f5c7..c44bad7 100644 --- a/src/java/org/apache/cassandra/dht/BootStrapper.java +++ b/src/java/org/apache/cassandra/dht/BootStrapper.java @@ -38,7 +38,6 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.TokenMetadata; -import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.*; import org.apache.cassandra.utils.progress.ProgressEvent; import org.apache.cassandra.utils.progress.ProgressEventNotifierSupport; @@ -77,7 +76,7 @@ public class BootStrapper extends ProgressEventNotifierSupport stateStore); streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance)); - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { AbstractReplicationStrategy strategy = Keyspace.open(keyspaceName).getReplicationStrategy(); streamer.addRanges(keyspaceName, strategy.getPendingAddressRanges(tokenMetadata, tokens, address)); http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java index 2276c4a..01122af 100644 --- a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java +++ b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java @@ -26,6 +26,7 @@ import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.List; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; @@ -56,12 +57,13 @@ public class PendingRangeCalculatorService public void run() { long start = System.currentTimeMillis(); - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + List keyspaces = Schema.instance.getNonLocalStrategyKeyspaces(); + for (String keyspaceName : keyspaces) { calculatePendingRanges(Keyspace.open(keyspaceName).getReplicationStrategy(), keyspaceName); } PendingRangeCalculatorService.instance.finishUpdate(); - logger.debug("finished calculation for {} keyspaces in {}ms", Schema.instance.getNonSystemKeyspaces().size(), System.currentTimeMillis() - start); + logger.debug("finished calculation for {} keyspaces in {}ms", keyspaces.size(), System.currentTimeMillis() - start); } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/service/StorageService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index a1e5b6b..6fa130a 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1160,7 +1160,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE if (sourceDc != null) streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(DatabaseDescriptor.getEndpointSnitch(), sourceDc)); - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) streamer.addRanges(keyspaceName, getLocalRanges(keyspaceName)); StreamResultFuture resultFuture = streamer.fetchAsync(); @@ -1449,7 +1449,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE // some people just want to get a visual representation of things. Allow null and set it to the first // non-system keyspace. if (keyspace == null) - keyspace = Schema.instance.getNonSystemKeyspaces().get(0); + keyspace = Schema.instance.getNonLocalStrategyKeyspaces().get(0); Map, List> map = new HashMap<>(); for (Map.Entry, Collection> entry : tokenMetadata.getPendingRangesMM(keyspace).asMap().entrySet()) @@ -1510,7 +1510,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE // some people just want to get a visual representation of things. Allow null and set it to the first // non-system keyspace. if (keyspace == null) - keyspace = Schema.instance.getNonSystemKeyspaces().get(0); + keyspace = Schema.instance.getNonLocalStrategyKeyspaces().get(0); List> ranges = getAllRanges(sortedTokens); return constructRangeToEndpointMap(keyspace, ranges); @@ -2348,7 +2348,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE InetAddress myAddress = FBUtilities.getBroadcastAddress(); - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { Multimap, InetAddress> changedRanges = getChangedRangesForLeaving(keyspaceName, endpoint); Set> myNewRanges = new HashSet<>(); @@ -3519,7 +3519,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE throw new UnsupportedOperationException("Node in " + operationMode + " state; wait for status to become normal or restart"); PendingRangeCalculatorService.instance.blockUntilFinished(); - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { if (tokenMetadata.getPendingRanges(keyspaceName, FBUtilities.getBroadcastAddress()).size() > 0) throw new UnsupportedOperationException("data is currently moving to this node; unable to leave the ring"); @@ -3568,7 +3568,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE { Map, InetAddress>> rangesToStream = new HashMap<>(); - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { Multimap, InetAddress> rangesMM = getChangedRangesForLeaving(keyspaceName, FBUtilities.getBroadcastAddress()); @@ -3687,7 +3687,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE throw new UnsupportedOperationException("This node has more than one token and cannot be moved thusly."); } - List keyspacesToProcess = Schema.instance.getNonSystemKeyspaces(); + List keyspacesToProcess = Schema.instance.getNonLocalStrategyKeyspaces(); PendingRangeCalculatorService.instance.blockUntilFinished(); // checking if data is moving to this node @@ -3945,7 +3945,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE throw new UnsupportedOperationException("This node is already processing a removal. Wait for it to complete, or use 'removenode force' if this has failed."); // Find the endpoints that are going to become responsible for data - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { // if the replication factor is 1 the data is lost so we shouldn't wait for confirmation if (Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor() == 1) @@ -4257,11 +4257,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE return Collections.unmodifiableList(keyspaceNamesList); } - public List getNonSystemKeyspaces() { - List keyspaceNamesList = new ArrayList<>(Schema.instance.getNonSystemKeyspaces()); - return Collections.unmodifiableList(keyspaceNamesList); + return Collections.unmodifiableList(Schema.instance.getNonSystemKeyspaces()); + } + + public List getNonLocalStrategyKeyspaces() + { + return Collections.unmodifiableList(Schema.instance.getNonLocalStrategyKeyspaces()); } public void updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException @@ -4481,7 +4484,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE public List sampleKeyRange() // do not rename to getter - see CASSANDRA-4452 for details { List keys = new ArrayList<>(); - for (Keyspace keyspace : Keyspace.nonSystem()) + for (Keyspace keyspace : Keyspace.nonLocalStrategy()) { for (Range range : getPrimaryRangesForEndpoint(keyspace.getName(), FBUtilities.getBroadcastAddress())) keys.addAll(keySamples(keyspace.getColumnFamilyStores(), range)); http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/service/StorageServiceMBean.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 5b7331d..e111bc4 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -446,6 +446,8 @@ public interface StorageServiceMBean extends NotificationEmitter public List getNonSystemKeyspaces(); + public List getNonLocalStrategyKeyspaces(); + /** * Change endpointsnitch class and dynamic-ness (and dynamic attributes) at runtime * @param epSnitchClassName the canonical path name for a class implementing IEndpointSnitch http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/tools/NodeProbe.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index a9d71d8..e1388f1 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -818,6 +818,11 @@ public class NodeProbe implements AutoCloseable return ssProxy.getNonSystemKeyspaces(); } + public List getNonLocalStrategyKeyspaces() + { + return ssProxy.getNonLocalStrategyKeyspaces(); + } + public String getClusterName() { return ssProxy.getClusterName(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/tools/NodeTool.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 5067a13..7d125ad 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -315,19 +315,34 @@ public class NodeTool return nodeClient; } + protected enum KeyspaceSet + { + ALL, NON_SYSTEM, NON_LOCAL_STRATEGY + } + protected List parseOptionalKeyspace(List cmdArgs, NodeProbe nodeProbe) { - return parseOptionalKeyspace(cmdArgs, nodeProbe, false); + return parseOptionalKeyspace(cmdArgs, nodeProbe, KeyspaceSet.NON_SYSTEM); } - protected List parseOptionalKeyspace(List cmdArgs, NodeProbe nodeProbe, boolean includeSystemKS) + protected List parseOptionalKeyspace(List cmdArgs, NodeProbe nodeProbe, KeyspaceSet defaultKeyspaceSet) { List keyspaces = new ArrayList<>(); + if (cmdArgs == null || cmdArgs.isEmpty()) - keyspaces.addAll(includeSystemKS ? nodeProbe.getKeyspaces() : nodeProbe.getNonSystemKeyspaces()); + { + if (defaultKeyspaceSet == KeyspaceSet.NON_LOCAL_STRATEGY) + keyspaces.addAll(keyspaces = nodeProbe.getNonLocalStrategyKeyspaces()); + else if (defaultKeyspaceSet == KeyspaceSet.NON_SYSTEM) + keyspaces.addAll(keyspaces = nodeProbe.getNonSystemKeyspaces()); + else + keyspaces.addAll(nodeProbe.getKeyspaces()); + } else + { keyspaces.add(cmdArgs.get(0)); + } for (String keyspace : keyspaces) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java index a7507b1..41e9b01 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java @@ -42,7 +42,7 @@ public class Cleanup extends NodeToolCmd @Override public void execute(NodeProbe probe) { - List keyspaces = parseOptionalKeyspace(args, probe); + List keyspaces = parseOptionalKeyspace(args, probe, KeyspaceSet.NON_LOCAL_STRATEGY); String[] tableNames = parseOptionalTables(args); for (String keyspace : keyspaces) http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/src/java/org/apache/cassandra/tools/nodetool/Repair.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/tools/nodetool/Repair.java b/src/java/org/apache/cassandra/tools/nodetool/Repair.java index bd1a916..02bfc5b 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Repair.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Repair.java @@ -84,7 +84,7 @@ public class Repair extends NodeToolCmd @Override public void execute(NodeProbe probe) { - List keyspaces = parseOptionalKeyspace(args, probe); + List keyspaces = parseOptionalKeyspace(args, probe, KeyspaceSet.NON_LOCAL_STRATEGY); String[] cfnames = parseOptionalTables(args); if (primaryRange && (!specificDataCenters.isEmpty() || !specificHosts.isEmpty())) http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/test/unit/org/apache/cassandra/dht/BootStrapperTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java index fbef538..8974791 100644 --- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java +++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java @@ -77,7 +77,7 @@ public class BootStrapperTest public void testSourceTargetComputation() throws UnknownHostException { final int[] clusterSizes = new int[] { 1, 3, 5, 10, 100}; - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { int replicationFactor = Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor(); for (int clusterSize : clusterSizes) http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java index 34338ea..0955985 100644 --- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java +++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java @@ -94,7 +94,7 @@ public class SimpleStrategyTest { TokenMetadata tmd; AbstractReplicationStrategy strategy; - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { tmd = new TokenMetadata(); strategy = getStrategy(keyspaceName, tmd); @@ -149,7 +149,7 @@ public class SimpleStrategyTest tmd.addBootstrapToken(bsToken, bootstrapEndpoint); AbstractReplicationStrategy strategy = null; - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { strategy = getStrategy(keyspaceName, tmd); http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java index b2bb081..efab615 100644 --- a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java +++ b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java @@ -96,7 +96,7 @@ public class LeaveAndBootstrapTest Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE); Map> expectedEndpoints = new HashMap>(); - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { for (Token token : keyTokens) { @@ -120,7 +120,7 @@ public class LeaveAndBootstrapTest PendingRangeCalculatorService.instance.blockUntilFinished(); AbstractReplicationStrategy strategy; - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { strategy = getStrategy(keyspaceName, tmd); for (Token token : keyTokens) http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e11828/test/unit/org/apache/cassandra/service/MoveTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java b/test/unit/org/apache/cassandra/service/MoveTest.java index 280c4e6..53365aa 100644 --- a/test/unit/org/apache/cassandra/service/MoveTest.java +++ b/test/unit/org/apache/cassandra/service/MoveTest.java @@ -502,7 +502,7 @@ public class MoveTest private void assertPendingRanges(TokenMetadata tmd, Map, Collection> pendingRanges, String keyspaceName) throws ConfigurationException { boolean keyspaceFound = false; - for (String nonSystemKeyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String nonSystemKeyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { if(!keyspaceName.equals(nonSystemKeyspaceName)) continue; @@ -571,7 +571,7 @@ public class MoveTest assertTrue(tmd.isMoving(hosts.get(MOVING_NODE))); AbstractReplicationStrategy strategy; - for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) { strategy = getStrategy(keyspaceName, tmd); if(strategy instanceof NetworkTopologyStrategy)