This is an automated email from the ASF dual-hosted git repository.

mck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 2346ed8  Offline token allocation strategy generator tool
2346ed8 is described below

commit 2346ed8241022882e77433e283ab8ce3004d12b0
Author: Mick Semb Wever <[email protected]>
AuthorDate: Wed Dec 2 09:34:28 2020 +0100

    Offline token allocation strategy generator tool
    
    Provides the tools/bin/generate-tokens script that can be used to 
pregenerate allocation strategy tokens.
    Refactors TokenAllocation for extensibility (and better re-use between 
strategies), and adds OfflineTokenAllocator.
    TokenMetadata now has a configurable snitch (instead of hardcoded to 
DatabaseDescriptor.getEndpointSnitch()) for testability.
    Adds logging of growth and stddev changes to 
allocate_tokens_for_local_replication_factor usages.
    
     patch by Mick Semb Wever; reviewed by Paul Motta, Ekaterina Dimitrova for 
CASSANDRA-16205
---
 CHANGES.txt                                        |   1 +
 debian/cassandra-tools.install                     |   3 +-
 doc/source/tools/generatetokens.rst                |  56 +++
 doc/source/tools/index.rst                         |   1 +
 redhat/cassandra.spec                              |   2 +
 .../dht/tokenallocator/OfflineTokenAllocator.java  | 190 +++++++++++
 .../dht/tokenallocator/TokenAllocation.java        | 379 ++++++++++-----------
 .../locator/AbstractReplicationStrategy.java       |  10 -
 .../org/apache/cassandra/locator/SimpleSnitch.java |   4 +-
 .../apache/cassandra/locator/TokenMetadata.java    |  33 +-
 .../org/apache/cassandra/tools/GenerateTokens.java | 157 +++++++++
 .../org/apache/cassandra/utils/OutputHandler.java  |  13 +-
 .../org/apache/cassandra/dht/BootStrapperTest.java | 213 +-----------
 .../tokenallocator/OfflineTokenAllocatorTest.java  | 222 ++++++++++++
 .../TokenAllocationTest.java}                      | 306 +++++++----------
 .../apache/cassandra/tools/GenerateTokensTest.java |  56 +++
 tools/bin/generatetokens                           |  54 +++
 17 files changed, 1089 insertions(+), 611 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 172ebcc..52d2526 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-beta4
+ * Add generatetokens script for offline token allocation strategy generation 
(CASSANDRA-16205)
  * Remove Windows scripts (CASSANDRA-16171)
  * Improve checksumming and compression in protocol V5 (CASSANDRA-15299)
  * Optimised repair streaming improvements (CASSANDRA-16274)
diff --git a/debian/cassandra-tools.install b/debian/cassandra-tools.install
index 8806344..7cebbef 100644
--- a/debian/cassandra-tools.install
+++ b/debian/cassandra-tools.install
@@ -1,7 +1,8 @@
+tools/bin/generatetokens usr/bin
+tools/bin/sstabledump usr/bin
 tools/bin/sstableexpiredblockers usr/bin
 tools/bin/sstablelevelreset usr/bin
 tools/bin/sstablemetadata usr/bin
 tools/bin/sstableofflinerelevel usr/bin
 tools/bin/sstablerepairedset usr/bin
 tools/bin/sstablesplit usr/bin
-tools/bin/sstabledump usr/bin
diff --git a/doc/source/tools/generatetokens.rst 
b/doc/source/tools/generatetokens.rst
new file mode 100644
index 0000000..24448d0
--- /dev/null
+++ b/doc/source/tools/generatetokens.rst
@@ -0,0 +1,56 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+
+generatetokens
+------------
+
+Pre-generates tokens for a datacenter with the given number of nodes using the 
token allocation algorithm. Useful in edge-cases when generated tokens needs to 
be known in advance of bootstrapping nodes. In nearly all cases it is best to 
just let the bootstrapping nodes automatically generate their own tokens.
+ref: https://issues.apache.org/jira/browse/CASSANDRA-16205
+
+
+Usage
+^^^^^
+generatetokens -n NODES -t TOKENS --rf REPLICATION_FACTOR [--partitioner 
PARTITIONER] [--racks RACK_NODE_COUNTS]
+
+
+===================================                   
================================================================================
+    -n,--nodes <arg>                                  Number of nodes.
+    -t,--tokens <arg>                                 Number of tokens/vnodes 
per node.
+    --rf <arg>                                        Replication factor.
+    -p,--partitioner <arg>                            Database partitioner, 
either Murmur3Partitioner or RandomPartitioner.
+    --racks <arg>                                     Number of nodes per 
rack, separated by commas. Must add up to the total node count. For example, 
'generatetokens -n 30 -t 8 --rf 3 --racks 10,10,10' will generate tokens for 
three racks of 10 nodes each.
+===================================                   
================================================================================
+
+
+This command, if used, is expected to be run before the Cassandra node is 
first started. The output from the command is used to configure the nodes 
`num_tokens` setting in the `cassandra.yaml`
+
+
+Example Output
+^^^^^^^^^^^^^^
+Example usage and output is
+
+    $ tools/bin/generatetokens -n 9 -t 4 --rf 3 --racks 3,3,3
+
+    Generating tokens for 9 nodes with 4 vnodes each for replication factor 3 
and partitioner Murmur3Partitioner
+    Node 0 rack 0: [-6270077235120413733, -1459727275878514299, 
2887564907718879562, 5778609289102954400]
+    Node 1 rack 1: [-8780789453057732897, -3279530982831298765, 
1242905755717369197, 8125606440590916903]
+    Node 2 rack 2: [-7240625170832344686, -4453155190605073029, 
74749827226930055, 4615117940688406403]
+    Node 3 rack 0: [-5361616212862743381, -2369629129354906532, 
2065235331718124379, 6952107864846935651]
+    Node 4 rack 1: [-8010707311945038792, -692488724325792122, 
3751341424203642982, 7538857152718926277]
+    Node 5 rack 2: [-7625666241388691739, -3866343086718185897, 
5196863614895680401, 8895780530621367810]
+    Node 6 rack 0: [-5815846723991578557, -1076108000102153211, 
1654070543717746788, 8510693485606142356]
+    Node 7 rack 1: [-2824580056093102649, 658827791472149626, 
3319453165961261272, 6365358576974945025]
+    Node 8 rack 2: [-4159749138661629463, -1914678202616710416, 
4905990777792043402, 6658733220910940338]
\ No newline at end of file
diff --git a/doc/source/tools/index.rst b/doc/source/tools/index.rst
index d28929c..56fbabc 100644
--- a/doc/source/tools/index.rst
+++ b/doc/source/tools/index.rst
@@ -23,6 +23,7 @@ This section describes the command line tools provided with 
Apache Cassandra.
    :maxdepth: 3
 
    cqlsh
+   generatetokens
    nodetool/nodetool
    sstable/index
    cassandra_stress
diff --git a/redhat/cassandra.spec b/redhat/cassandra.spec
index f0652c4..dcbda87 100644
--- a/redhat/cassandra.spec
+++ b/redhat/cassandra.spec
@@ -126,6 +126,7 @@ exit 0
 %attr(755,root,root) %{_bindir}/cqlsh.py
 %attr(755,root,root) %{_bindir}/debug-cql
 %attr(755,root,root) %{_bindir}/fqltool
+%attr(755,root,root) %{_bindir}/generatetokens
 %attr(755,root,root) %{_bindir}/nodetool
 %attr(755,root,root) %{_bindir}/sstableloader
 %attr(755,root,root) %{_bindir}/sstablescrub
@@ -180,6 +181,7 @@ This package contains extra tools for working with 
Cassandra clusters.
 %attr(755,root,root) %{_bindir}/auditlogviewer
 %attr(755,root,root) %{_bindir}/jmxtool
 %attr(755,root,root) %{_bindir}/fqltool
+%attr(755,root,root) %{_bindir}/generatetokens
 
 
 %changelog
diff --git 
a/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java 
b/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java
new file mode 100644
index 0000000..f8c13df
--- /dev/null
+++ 
b/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.dht.tokenallocator;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeSet;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
+
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.SimpleSnitch;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class OfflineTokenAllocator
+{
+    public static List<FakeNode> allocate(int rf, int numTokens, int[] 
nodesPerRack, OutputHandler logger, IPartitioner partitioner)
+    {
+        Preconditions.checkArgument(rf > 0, "rf must be greater than zero");
+        Preconditions.checkArgument(numTokens > 0, "num_tokens must be greater 
than zero");
+        Preconditions.checkNotNull(nodesPerRack);
+        Preconditions.checkArgument(nodesPerRack.length > 0, "nodesPerRack 
must contain a node count for at least one rack");
+        Preconditions.checkNotNull(logger);
+        Preconditions.checkNotNull(partitioner);
+
+        int nodes = Arrays.stream(nodesPerRack).sum();
+
+        Preconditions.checkArgument(nodes >= rf,
+                                    "not enough nodes %s for rf %s in %s", 
Arrays.stream(nodesPerRack).sum(), rf, Arrays.toString(nodesPerRack));
+
+        List<FakeNode> fakeNodes = new ArrayList<>(nodes);
+        MultinodeAllocator allocator = new MultinodeAllocator(rf, numTokens, 
logger, partitioner);
+
+        // Defensive-copy method argument
+        nodesPerRack = Arrays.copyOf(nodesPerRack, nodesPerRack.length);
+
+        int racks = nodesPerRack.length;
+        int nodeId = 0;
+        int rackId = 0;
+        // Distribute nodes among the racks in round-robin fashion in the 
order the user is supposed to start them.
+        while (nodesPerRack[rackId] > 0)
+        {
+            // Allocate tokens for current node
+            fakeNodes.add(allocator.allocateTokensForNode(nodeId++, rackId));
+
+            // Find next rack with unallocated node
+            int nextRack = (rackId+1) % racks;
+            while (nodesPerRack[nextRack] == 0 && nextRack != rackId)
+                nextRack = (nextRack+1) % racks;
+
+            // Update nodesPerRack and rackId
+            nodesPerRack[rackId]--;
+            rackId = nextRack;
+        }
+        return fakeNodes;
+    }
+
+    public static class FakeNode
+    {
+        private final InetAddressAndPort fakeAddressAndPort;
+        private final int rackId;
+        private final Collection<Token> tokens;
+
+        public FakeNode(InetAddressAndPort address, Integer rackId, 
Collection<Token> tokens)
+        {
+            this.fakeAddressAndPort = address;
+            this.rackId = rackId;
+            // Sort tokens for better presentation
+            this.tokens = new TreeSet<>(tokens);
+        }
+
+        public int nodeId()
+        {
+            return fakeAddressAndPort.port;
+        }
+
+        public int rackId()
+        {
+            return rackId;
+        }
+
+        public Collection<Token> tokens()
+        {
+            return tokens;
+        }
+    }
+
+    private static class MultinodeAllocator
+    {
+        private final FakeSnitch fakeSnitch;
+        private final TokenMetadata fakeMetadata;
+        private final TokenAllocation allocation;
+        private final Map<Integer, SummaryStatistics> lastCheckPoint = 
Maps.newHashMap();
+        private final OutputHandler logger;
+
+        private MultinodeAllocator(int rf, int numTokens, OutputHandler 
logger, IPartitioner partitioner)
+        {
+            this.fakeSnitch = new FakeSnitch();
+            this.fakeMetadata = new 
TokenMetadata(fakeSnitch).cloneWithNewPartitioner(partitioner);
+            this.allocation = TokenAllocation.create(fakeSnitch, fakeMetadata, 
rf, numTokens);
+            this.logger = logger;
+        }
+
+        private FakeNode allocateTokensForNode(int nodeId, Integer rackId)
+        {
+            // Update snitch and token metadata info
+            InetAddressAndPort fakeNodeAddressAndPort = 
getLoopbackAddressWithPort(nodeId);
+            fakeSnitch.nodeByRack.put(fakeNodeAddressAndPort, rackId);
+            fakeMetadata.updateTopology(fakeNodeAddressAndPort);
+
+            // Allocate tokens
+            Collection<Token> tokens = 
allocation.allocate(fakeNodeAddressAndPort);
+
+            // Validate ownership stats
+            validateAllocation(nodeId, rackId);
+
+            return new FakeNode(fakeNodeAddressAndPort, rackId, tokens);
+        }
+
+        private void validateAllocation(int nodeId, int rackId)
+        {
+            SummaryStatistics newOwnership = 
allocation.getAllocationRingOwnership(SimpleSnitch.DATA_CENTER_NAME, 
Integer.toString(rackId));
+            SummaryStatistics oldOwnership = lastCheckPoint.put(rackId, 
newOwnership);
+            if (oldOwnership != null)
+                logger.debug(String.format("Replicated node load in rack=%d 
before allocating node %d: %s.", rackId, nodeId,
+                                           
TokenAllocation.statToString(oldOwnership)));
+            logger.debug(String.format("Replicated node load in rack=%d after 
allocating node %d: %s.", rackId, nodeId,
+                                       
TokenAllocation.statToString(newOwnership)));
+            if (oldOwnership != null && oldOwnership.getStandardDeviation() != 
0.0)
+            {
+                double stdDevGrowth = newOwnership.getStandardDeviation() - 
oldOwnership.getStandardDeviation();
+                if (stdDevGrowth > TokenAllocation.WARN_STDEV_GROWTH)
+                {
+                    logger.warn(String.format("Growth of %.2f%% in token 
ownership standard deviation after allocating node %d on rack %d above warning 
threshold of %d%%",
+                                              stdDevGrowth * 100, nodeId, 
rackId, (int)(TokenAllocation.WARN_STDEV_GROWTH * 100)));
+                }
+            }
+        }
+    }
+
+    private static class FakeSnitch extends SimpleSnitch
+    {
+        final Map<InetAddressAndPort, Integer> nodeByRack = new HashMap<>();
+
+        @Override
+        public String getRack(InetAddressAndPort endpoint)
+        {
+            return Integer.toString(nodeByRack.get(endpoint));
+        }
+    }
+
+    private static InetAddressAndPort getLoopbackAddressWithPort(int port)
+    {
+        try
+        {
+            return 
InetAddressAndPort.getByAddressOverrideDefaults(InetAddress.getByName("127.0.0.1"),
 port);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new IllegalStateException("Unexpected UnknownHostException", 
e);
+        }
+    }
+}
\ No newline at end of file
diff --git 
a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java 
b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
index bd6d980..e3e81dc 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.dht.tokenallocator;
 
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -45,32 +46,27 @@ import org.apache.cassandra.locator.TokenMetadata.Topology;
 
 public class TokenAllocation
 {
+    public static final double WARN_STDEV_GROWTH = 0.05;
+
     private static final Logger logger = 
LoggerFactory.getLogger(TokenAllocation.class);
+    final TokenMetadata tokenMetadata;
+    final AbstractReplicationStrategy replicationStrategy;
+    final int numTokens;
+    final Map<String, Map<String, StrategyAdapter>> strategyByRackDc = new 
HashMap<>();
+
+    private TokenAllocation(TokenMetadata tokenMetadata, 
AbstractReplicationStrategy replicationStrategy, int numTokens)
+    {
+        this.tokenMetadata = tokenMetadata.cloneOnlyTokenMap();
+        this.replicationStrategy = replicationStrategy;
+        this.numTokens = numTokens;
+    }
 
     public static Collection<Token> allocateTokens(final TokenMetadata 
tokenMetadata,
                                                    final 
AbstractReplicationStrategy rs,
                                                    final InetAddressAndPort 
endpoint,
                                                    int numTokens)
     {
-        TokenMetadata tokenMetadataCopy = tokenMetadata.cloneOnlyTokenMap();
-        StrategyAdapter strategy = getStrategy(tokenMetadataCopy, rs, 
endpoint);
-        Collection<Token> tokens = create(tokenMetadata, 
strategy).addUnit(endpoint, numTokens);
-        tokens = adjustForCrossDatacenterClashes(tokenMetadata, strategy, 
tokens);
-
-        if (logger.isWarnEnabled())
-        {
-            logger.warn("Selected tokens {}", tokens);
-            SummaryStatistics os = replicatedOwnershipStats(tokenMetadataCopy, 
rs, endpoint);
-            tokenMetadataCopy.updateNormalTokens(tokens, endpoint);
-            SummaryStatistics ns = replicatedOwnershipStats(tokenMetadataCopy, 
rs, endpoint);
-            logger.warn("Replicated node load in datacenter before allocation 
{}", statToString(os));
-            logger.warn("Replicated node load in datacenter after allocation 
{}", statToString(ns));
-
-            // TODO: Is it worth doing the replicated ownership calculation 
always to be able to raise this alarm?
-            if (ns.getStandardDeviation() > os.getStandardDeviation())
-                logger.warn("Unexpected growth in standard deviation after 
allocation.");
-        }
-        return tokens;
+        return create(tokenMetadata, rs, numTokens).allocate(endpoint);
     }
 
     public static Collection<Token> allocateTokens(final TokenMetadata 
tokenMetadata,
@@ -78,260 +74,229 @@ public class TokenAllocation
                                                    final InetAddressAndPort 
endpoint,
                                                    int numTokens)
     {
-        TokenMetadata tokenMetadataCopy = tokenMetadata.cloneOnlyTokenMap();
-        StrategyAdapter strategy = getStrategy(tokenMetadataCopy, replicas, 
endpoint);
-        Collection<Token> tokens = create(tokenMetadata, 
strategy).addUnit(endpoint, numTokens);
-        tokens = adjustForCrossDatacenterClashes(tokenMetadata, strategy, 
tokens);
-        logger.warn("Selected tokens {}", tokens);
-        // SummaryStatistics is not implemented for 
`allocate_tokens_for_local_replication_factor`
-        return tokens;
+        return create(DatabaseDescriptor.getEndpointSnitch(), tokenMetadata, 
replicas, numTokens).allocate(endpoint);
     }
 
-    private static Collection<Token> adjustForCrossDatacenterClashes(final 
TokenMetadata tokenMetadata,
-                                                                     
StrategyAdapter strategy, Collection<Token> tokens)
+    static TokenAllocation create(IEndpointSnitch snitch, TokenMetadata 
tokenMetadata, int replicas, int numTokens)
     {
-        List<Token> filtered = Lists.newArrayListWithCapacity(tokens.size());
+        // We create a fake NTS replication strategy with the specified RF in 
the local DC
+        HashMap<String, String> options = new HashMap<>();
+        options.put(snitch.getLocalDatacenter(), Integer.toString(replicas));
+        NetworkTopologyStrategy fakeReplicationStrategy = new 
NetworkTopologyStrategy(null, tokenMetadata, snitch, options);
 
-        for (Token t : tokens)
-        {
-            while (tokenMetadata.getEndpoint(t) != null)
-            {
-                InetAddressAndPort other = tokenMetadata.getEndpoint(t);
-                if (strategy.inAllocationRing(other))
-                    throw new ConfigurationException(String.format("Allocated 
token %s already assigned to node %s. Is another node also allocating tokens?", 
t, other));
-                t = t.increaseSlightly();
-            }
-            filtered.add(t);
-        }
-        return filtered;
+        TokenAllocation allocator = new TokenAllocation(tokenMetadata, 
fakeReplicationStrategy, numTokens);
+        return allocator;
     }
 
-    // return the ratio of ownership for each endpoint
-    public static Map<InetAddressAndPort, Double> 
evaluateReplicatedOwnership(TokenMetadata tokenMetadata, 
AbstractReplicationStrategy rs)
+    static TokenAllocation create(TokenMetadata tokenMetadata, 
AbstractReplicationStrategy rs, int numTokens)
     {
-        Map<InetAddressAndPort, Double> ownership = Maps.newHashMap();
-        List<Token> sortedTokens = tokenMetadata.sortedTokens();
-        Iterator<Token> it = sortedTokens.iterator();
-        Token current = it.next();
-        while (it.hasNext())
-        {
-            Token next = it.next();
-            addOwnership(tokenMetadata, rs, current, next, ownership);
-            current = next;
-        }
-        addOwnership(tokenMetadata, rs, current, sortedTokens.get(0), 
ownership);
-
-        return ownership;
+        return new TokenAllocation(tokenMetadata, rs, numTokens);
     }
 
-    static void addOwnership(final TokenMetadata tokenMetadata, final 
AbstractReplicationStrategy rs, Token current, Token next, 
Map<InetAddressAndPort, Double> ownership)
+    Collection<Token> allocate(InetAddressAndPort endpoint)
     {
-        double size = current.size(next);
-        Token representative = current.getPartitioner().midpoint(current, 
next);
-        for (InetAddressAndPort n : 
rs.calculateNaturalReplicas(representative, tokenMetadata).endpoints())
+        StrategyAdapter strategy = getOrCreateStrategy(endpoint);
+        Collection<Token> tokens = 
strategy.createAllocator().addUnit(endpoint, numTokens);
+        tokens = strategy.adjustForCrossDatacenterClashes(tokens);
+
+        SummaryStatistics os = strategy.replicatedOwnershipStats();
+        tokenMetadata.updateNormalTokens(tokens, endpoint);
+
+        SummaryStatistics ns = strategy.replicatedOwnershipStats();
+        logger.info("Selected tokens {}", tokens);
+        logger.debug("Replicated node load in datacenter before allocation 
{}", statToString(os));
+        logger.debug("Replicated node load in datacenter after allocation {}", 
statToString(ns));
+
+        double stdDevGrowth = ns.getStandardDeviation() - 
os.getStandardDeviation();
+        if (stdDevGrowth > TokenAllocation.WARN_STDEV_GROWTH)
         {
-            Double v = ownership.get(n);
-            ownership.put(n, v != null ? v + size : size);
+            logger.warn(String.format("Growth of %.2f%% in token ownership 
standard deviation after allocation above warning threshold of %d%%",
+                                      stdDevGrowth * 100, 
(int)(TokenAllocation.WARN_STDEV_GROWTH * 100)));
         }
+
+        return tokens;
     }
 
-    public static String statToString(SummaryStatistics stat)
+    static String statToString(SummaryStatistics stat)
     {
         return String.format("max %.2f min %.2f stddev %.4f", stat.getMax() / 
stat.getMean(), stat.getMin() / stat.getMean(), stat.getStandardDeviation());
     }
 
-    public static SummaryStatistics replicatedOwnershipStats(TokenMetadata 
tokenMetadata,
-                                                             
AbstractReplicationStrategy rs, InetAddressAndPort endpoint)
+    SummaryStatistics getAllocationRingOwnership(String datacenter, String 
rack)
     {
-        SummaryStatistics stat = new SummaryStatistics();
-        StrategyAdapter strategy = getStrategy(tokenMetadata, rs, endpoint);
-        for (Map.Entry<InetAddressAndPort, Double> en : 
evaluateReplicatedOwnership(tokenMetadata, rs).entrySet())
-        {
-            // Filter only in the same datacentre.
-            if (strategy.inAllocationRing(en.getKey()))
-                stat.addValue(en.getValue() / 
tokenMetadata.getTokens(en.getKey()).size());
-        }
-        return stat;
+        return getOrCreateStrategy(datacenter, 
rack).replicatedOwnershipStats();
     }
 
-    static TokenAllocator<InetAddressAndPort> create(TokenMetadata 
tokenMetadata, StrategyAdapter strategy)
+    SummaryStatistics getAllocationRingOwnership(InetAddressAndPort endpoint)
     {
-        NavigableMap<Token, InetAddressAndPort> sortedTokens = new TreeMap<>();
-        for (Map.Entry<Token, InetAddressAndPort> en : 
tokenMetadata.getNormalAndBootstrappingTokenToEndpointMap().entrySet())
-        {
-            if (strategy.inAllocationRing(en.getValue()))
-                sortedTokens.put(en.getKey(), en.getValue());
-        }
-        return TokenAllocatorFactory.createTokenAllocator(sortedTokens, 
strategy, tokenMetadata.partitioner);
+        return getOrCreateStrategy(endpoint).replicatedOwnershipStats();
     }
 
-    interface StrategyAdapter extends ReplicationStrategy<InetAddressAndPort>
+    abstract class StrategyAdapter implements 
ReplicationStrategy<InetAddressAndPort>
     {
         // return true iff the provided endpoint occurs in the same virtual 
token-ring we are allocating for
         // i.e. the set of the nodes that share ownership with the node we are 
allocating
         // alternatively: return false if the endpoint's ownership is 
independent of the node we are allocating tokens for
-        boolean inAllocationRing(InetAddressAndPort other);
-    }
+        abstract boolean inAllocationRing(InetAddressAndPort other);
 
-    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, 
final AbstractReplicationStrategy rs, final InetAddressAndPort endpoint)
-    {
-        if (rs instanceof NetworkTopologyStrategy)
-            return getStrategy(tokenMetadata, (NetworkTopologyStrategy) rs, 
rs.snitch, endpoint);
-        if (rs instanceof SimpleStrategy)
-            return getStrategy(tokenMetadata, (SimpleStrategy) rs, endpoint);
-        throw new ConfigurationException("Token allocation does not support 
replication strategy " + rs.getClass().getSimpleName());
-    }
+        final TokenAllocator<InetAddressAndPort> createAllocator()
+        {
+            NavigableMap<Token, InetAddressAndPort> sortedTokens = new 
TreeMap<>();
+            for (Map.Entry<Token, InetAddressAndPort> en : 
tokenMetadata.getNormalAndBootstrappingTokenToEndpointMap().entrySet())
+            {
+                if (inAllocationRing(en.getValue()))
+                    sortedTokens.put(en.getKey(), en.getValue());
+            }
+            return TokenAllocatorFactory.createTokenAllocator(sortedTokens, 
this, tokenMetadata.partitioner);
+        }
 
-    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, 
final SimpleStrategy rs, final InetAddressAndPort endpoint)
-    {
-        final int replicas = rs.getReplicationFactor().allReplicas;
+        final Collection<Token> 
adjustForCrossDatacenterClashes(Collection<Token> tokens)
+        {
+            List<Token> filtered = 
Lists.newArrayListWithCapacity(tokens.size());
 
-        return new StrategyAdapter()
+            for (Token t : tokens)
+            {
+                while (tokenMetadata.getEndpoint(t) != null)
+                {
+                    InetAddressAndPort other = tokenMetadata.getEndpoint(t);
+                    if (inAllocationRing(other))
+                        throw new 
ConfigurationException(String.format("Allocated token %s already assigned to 
node %s. Is another node also allocating tokens?", t, other));
+                    t = t.increaseSlightly();
+                }
+                filtered.add(t);
+            }
+            return filtered;
+        }
+
+        final SummaryStatistics replicatedOwnershipStats()
         {
-            @Override
-            public int replicas()
+            SummaryStatistics stat = new SummaryStatistics();
+            for (Map.Entry<InetAddressAndPort, Double> en : 
evaluateReplicatedOwnership().entrySet())
             {
-                return replicas;
+                // Filter only in the same allocation ring
+                if (inAllocationRing(en.getKey()))
+                    stat.addValue(en.getValue() / 
tokenMetadata.getTokens(en.getKey()).size());
             }
+            return stat;
+        }
 
-            @Override
-            public Object getGroup(InetAddressAndPort unit)
+        // return the ratio of ownership for each endpoint
+        private Map<InetAddressAndPort, Double> evaluateReplicatedOwnership()
+        {
+            Map<InetAddressAndPort, Double> ownership = Maps.newHashMap();
+            List<Token> sortedTokens = tokenMetadata.sortedTokens();
+            if (sortedTokens.isEmpty())
+                return ownership;
+
+            Iterator<Token> it = sortedTokens.iterator();
+            Token current = it.next();
+            while (it.hasNext())
             {
-                return unit;
+                Token next = it.next();
+                addOwnership(current, next, ownership);
+                current = next;
             }
+            addOwnership(current, sortedTokens.get(0), ownership);
 
-            @Override
-            public boolean inAllocationRing(InetAddressAndPort other)
+            return ownership;
+        }
+
+        private void addOwnership(Token current, Token next, 
Map<InetAddressAndPort, Double> ownership)
+        {
+            double size = current.size(next);
+            Token representative = current.getPartitioner().midpoint(current, 
next);
+            for (InetAddressAndPort n : 
replicationStrategy.calculateNaturalReplicas(representative, 
tokenMetadata).endpoints())
             {
-                return true;
+                Double v = ownership.get(n);
+                ownership.put(n, v != null ? v + size : size);
             }
-        };
+        }
     }
 
-    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, 
final NetworkTopologyStrategy rs, final IEndpointSnitch snitch, final 
InetAddressAndPort endpoint)
+    private StrategyAdapter getOrCreateStrategy(InetAddressAndPort endpoint)
     {
-        final String dc = snitch.getDatacenter(endpoint);
-        final int replicas = rs.getReplicationFactor(dc).allReplicas;
-        return getStrategy(tokenMetadata, replicas, snitch, endpoint);
+        String dc = replicationStrategy.snitch.getDatacenter(endpoint);
+        String rack = replicationStrategy.snitch.getRack(endpoint);
+        return getOrCreateStrategy(dc, rack);
     }
 
-    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, 
final int replicas, final InetAddressAndPort endpoint)
+    private StrategyAdapter getOrCreateStrategy(String dc, String rack)
     {
-        return getStrategy(tokenMetadata, replicas, 
DatabaseDescriptor.getEndpointSnitch(), endpoint);
+        return strategyByRackDc.computeIfAbsent(dc, k -> new 
HashMap<>()).computeIfAbsent(rack, k -> createStrategy(dc, rack));
     }
 
-    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, 
final int replicas, final IEndpointSnitch snitch, final InetAddressAndPort 
endpoint)
+    private StrategyAdapter createStrategy(String dc, String rack)
     {
-        final String dc = snitch.getDatacenter(endpoint);
-        if (replicas == 0 || replicas == 1)
-        {
-            // No replication, each node is treated as separate.
-            return new StrategyAdapter()
-            {
-                @Override
-                public int replicas()
-                {
-                    return 1;
-                }
+        if (replicationStrategy instanceof NetworkTopologyStrategy)
+            return createStrategy(tokenMetadata, (NetworkTopologyStrategy) 
replicationStrategy, dc, rack);
+        if (replicationStrategy instanceof SimpleStrategy)
+            return createStrategy((SimpleStrategy) replicationStrategy);
+        throw new ConfigurationException("Token allocation does not support 
replication strategy " + replicationStrategy.getClass().getSimpleName());
+    }
 
-                @Override
-                public Object getGroup(InetAddressAndPort unit)
-                {
-                    return unit;
-                }
+    private StrategyAdapter createStrategy(final SimpleStrategy rs)
+    {
+        return createStrategy(rs.snitch, null, null, 
rs.getReplicationFactor().allReplicas, false);
+    }
 
-                @Override
-                public boolean inAllocationRing(InetAddressAndPort other)
-                {
-                    return dc.equals(snitch.getDatacenter(other));
-                }
-            };
-        }
+    private StrategyAdapter createStrategy(TokenMetadata tokenMetadata, 
NetworkTopologyStrategy strategy, String dc, String rack)
+    {
+        int replicas = strategy.getReplicationFactor(dc).allReplicas;
 
         Topology topology = tokenMetadata.getTopology();
-
-        // if topology hasn't been setup yet for this endpoint+rack then treat 
it as a separate unit
-        int racks = topology.getDatacenterRacks().get(dc) != null && 
topology.getDatacenterRacks().get(dc).containsKey(snitch.getRack(endpoint))
+        // if topology hasn't been setup yet for this dc+rack then treat it as 
a separate unit
+        int racks = topology.getDatacenterRacks().get(dc) != null && 
topology.getDatacenterRacks().get(dc).containsKey(rack)
                 ? topology.getDatacenterRacks().get(dc).asMap().size()
                 : 1;
 
-        if (racks > replicas)
+        if (replicas <= 1)
         {
-            return new StrategyAdapter()
-            {
-                @Override
-                public int replicas()
-                {
-                    return replicas;
-                }
-
-                @Override
-                public Object getGroup(InetAddressAndPort unit)
-                {
-                    return snitch.getRack(unit);
-                }
-
-                @Override
-                public boolean inAllocationRing(InetAddressAndPort other)
-                {
-                    return dc.equals(snitch.getDatacenter(other));
-                }
-            };
+            // each node is treated as separate and replicates once
+            return createStrategy(strategy.snitch, dc, null, 1, false);
         }
         else if (racks == replicas)
         {
-            // When the number of racks is the same as the replication factor, 
everything must replicate exactly once
-            // in each rack. This is the same as having independent rings from 
each rack.
-            final String rack = snitch.getRack(endpoint);
-            return new StrategyAdapter()
-            {
-                @Override
-                public int replicas()
-                {
-                    return 1;
-                }
-
-                @Override
-                public Object getGroup(InetAddressAndPort unit)
-                {
-                    return unit;
-                }
-
-                @Override
-                public boolean inAllocationRing(InetAddressAndPort other)
-                {
-                    return dc.equals(snitch.getDatacenter(other)) && 
rack.equals(snitch.getRack(other));
-                }
-            };
+            // each node is treated as separate and replicates once, with 
separate allocation rings for each rack
+            return createStrategy(strategy.snitch, dc, rack, 1, false);
+        }
+        else if (racks > replicas)
+        {
+            // group by rack
+            return createStrategy(strategy.snitch, dc, null, replicas, true);
         }
         else if (racks == 1)
         {
-            // One rack, each node treated as separate.
-            return new StrategyAdapter()
+            return createStrategy(strategy.snitch, dc, null, replicas, false);
+        }
+
+        throw new ConfigurationException(String.format("Token allocation 
failed: the number of racks %d in datacenter %s is lower than its replication 
factor %d.",
+                                                       racks, dc, replicas));
+    }
+
+    // a null dc will always return true for inAllocationRing(..)
+    // a null rack will return true for inAllocationRing(..) for all nodes in 
the same dc
+    private StrategyAdapter createStrategy(IEndpointSnitch snitch, String dc, 
String rack, int replicas, boolean groupByRack)
+    {
+        return new StrategyAdapter()
+        {
+            @Override
+            public int replicas()
             {
-                @Override
-                public int replicas()
-                {
-                    return replicas;
-                }
+                return replicas;
+            }
 
-                @Override
-                public Object getGroup(InetAddressAndPort unit)
-                {
-                    return unit;
-                }
+            @Override
+            public Object getGroup(InetAddressAndPort unit)
+            {
+                return groupByRack ? snitch.getRack(unit) : unit;
+            }
 
-                @Override
-                public boolean inAllocationRing(InetAddressAndPort other)
-                {
-                    return dc.equals(snitch.getDatacenter(other));
-                }
-            };
-        }
-        else
-            throw new ConfigurationException(
-                    String.format("Token allocation failed: the number of 
racks %d in datacenter %s is lower than its replication factor %d.",
-                                  racks, dc, replicas));
+            @Override
+            public boolean inAllocationRing(InetAddressAndPort other)
+            {
+                return (dc == null || dc.equals(snitch.getDatacenter(other))) 
&& (rack == null || rack.equals(snitch.getRack(other)));
+            }
+        };
     }
 }
 
diff --git 
a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java 
b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index 8329606..d5aacd0 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -53,7 +53,6 @@ public abstract class AbstractReplicationStrategy
 
     @VisibleForTesting
     final String keyspaceName;
-    private Keyspace keyspace;
     public final Map<String, String> configOptions;
     private final TokenMetadata tokenMetadata;
 
@@ -64,14 +63,12 @@ public abstract class AbstractReplicationStrategy
 
     protected AbstractReplicationStrategy(String keyspaceName, TokenMetadata 
tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
     {
-        assert keyspaceName != null;
         assert snitch != null;
         assert tokenMetadata != null;
         this.tokenMetadata = tokenMetadata;
         this.snitch = snitch;
         this.configOptions = configOptions == null ? Collections.<String, 
String>emptyMap() : configOptions;
         this.keyspaceName = keyspaceName;
-        // lazy-initialize keyspace itself since we don't create them until 
after the replication strategies
     }
 
     private final Map<Token, EndpointsForRange> cachedReplicas = new 
NonBlockingHashMap<>();
@@ -204,13 +201,6 @@ public abstract class AbstractReplicationStrategy
         return resultResponseHandler;
     }
 
-    private Keyspace getKeyspace()
-    {
-        if (keyspace == null)
-            keyspace = Keyspace.open(keyspaceName);
-        return keyspace;
-    }
-
     /**
      * calculate the RF based on strategy_options. When overwriting, ensure 
that this get()
      *  is FAST, as this is called often.
diff --git a/src/java/org/apache/cassandra/locator/SimpleSnitch.java 
b/src/java/org/apache/cassandra/locator/SimpleSnitch.java
index d605b6e..f26375a 100644
--- a/src/java/org/apache/cassandra/locator/SimpleSnitch.java
+++ b/src/java/org/apache/cassandra/locator/SimpleSnitch.java
@@ -24,6 +24,8 @@ package org.apache.cassandra.locator;
  */
 public class SimpleSnitch extends AbstractEndpointSnitch
 {
+    public static final String DATA_CENTER_NAME = "datacenter1";
+
     public String getRack(InetAddressAndPort endpoint)
     {
         return "rack1";
@@ -31,7 +33,7 @@ public class SimpleSnitch extends AbstractEndpointSnitch
 
     public String getDatacenter(InetAddressAndPort endpoint)
     {
-        return "datacenter1";
+        return DATA_CENTER_NAME;
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java 
b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index ca8fd99..546f44e 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -24,12 +24,10 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
-
-import org.apache.cassandra.config.CassandraRelevantProperties;
-import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,6 +38,7 @@ import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.BiMultiValMap;
 import org.apache.cassandra.utils.Pair;
@@ -111,12 +110,20 @@ public class TokenMetadata
 
     public TokenMetadata()
     {
-        this(SortedBiMultiValMap.<Token, InetAddressAndPort>create(),
+        this(SortedBiMultiValMap.create(),
              HashBiMap.create(),
              Topology.empty(),
              DatabaseDescriptor.getPartitioner());
     }
 
+    public TokenMetadata(IEndpointSnitch snitch)
+    {
+        this(SortedBiMultiValMap.create(),
+             HashBiMap.create(),
+             Topology.builder(() -> snitch).build(),
+             DatabaseDescriptor.getPartitioner());
+    }
+
     private TokenMetadata(BiMultiValMap<Token, InetAddressAndPort> 
tokenToEndpointMap, BiMap<InetAddressAndPort, UUID> endpointsMap, Topology 
topology, IPartitioner partitioner)
     {
         this.tokenToEndpointMap = tokenToEndpointMap;
@@ -1364,6 +1371,7 @@ public class TokenMetadata
         private final ImmutableMap<String, ImmutableMultimap<String, 
InetAddressAndPort>> dcRacks;
         /** reverse-lookup map for endpoint to current known dc/rack 
assignment */
         private final ImmutableMap<InetAddressAndPort, Pair<String, String>> 
currentLocations;
+        private final Supplier<IEndpointSnitch> snitchSupplier;
 
         private Topology(Builder builder)
         {
@@ -1375,6 +1383,7 @@ public class TokenMetadata
             this.dcRacks = dcRackBuilder.build();
 
             this.currentLocations = 
ImmutableMap.copyOf(builder.currentLocations);
+            this.snitchSupplier = builder.snitchSupplier;
         }
 
         /**
@@ -1406,14 +1415,14 @@ public class TokenMetadata
             return new Builder(this);
         }
 
-        static Builder builder()
+        static Builder builder(Supplier<IEndpointSnitch> snitchSupplier)
         {
-            return new Builder();
+            return new Builder(snitchSupplier);
         }
 
         static Topology empty()
         {
-            return builder().build();
+            return builder(() -> 
DatabaseDescriptor.getEndpointSnitch()).build();
         }
 
         private static class Builder
@@ -1424,12 +1433,14 @@ public class TokenMetadata
             private final Map<String, Multimap<String, InetAddressAndPort>> 
dcRacks;
             /** reverse-lookup map for endpoint to current known dc/rack 
assignment */
             private final Map<InetAddressAndPort, Pair<String, String>> 
currentLocations;
+            private final Supplier<IEndpointSnitch> snitchSupplier;
 
-            Builder()
+            Builder(Supplier<IEndpointSnitch> snitchSupplier)
             {
                 this.dcEndpoints = HashMultimap.create();
                 this.dcRacks = new HashMap<>();
                 this.currentLocations = new HashMap<>();
+                this.snitchSupplier = snitchSupplier;
             }
 
             Builder(Topology from)
@@ -1441,6 +1452,7 @@ public class TokenMetadata
                     dcRacks.put(entry.getKey(), 
HashMultimap.create(entry.getValue()));
 
                 this.currentLocations = new HashMap<>(from.currentLocations);
+                this.snitchSupplier = from.snitchSupplier;
             }
 
             /**
@@ -1448,9 +1460,8 @@ public class TokenMetadata
              */
             Builder addEndpoint(InetAddressAndPort ep)
             {
-                IEndpointSnitch snitch = 
DatabaseDescriptor.getEndpointSnitch();
-                String dc = snitch.getDatacenter(ep);
-                String rack = snitch.getRack(ep);
+                String dc = snitchSupplier.get().getDatacenter(ep);
+                String rack = snitchSupplier.get().getRack(ep);
                 Pair<String, String> current = currentLocations.get(ep);
                 if (current != null)
                 {
diff --git a/src/java/org/apache/cassandra/tools/GenerateTokens.java 
b/src/java/org/apache/cassandra/tools/GenerateTokens.java
new file mode 100644
index 0000000..c03a4d0
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/GenerateTokens.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools;
+
+import java.util.Arrays;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+
+import org.slf4j.LoggerFactory;
+
+import ch.qos.logback.classic.Level;
+import ch.qos.logback.classic.Logger;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.tokenallocator.OfflineTokenAllocator;
+import org.apache.cassandra.dht.tokenallocator.TokenAllocation;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class GenerateTokens
+{
+    private static final String RF = "rf";
+    private static final String TOKENS = "tokens";
+    private static final String NODES = "nodes";
+    private static final String PARTITIONER = "partitioner";
+    private static final String RACKS = "racks";
+    private static final String VERBOSE = "verbose";
+
+    public static void main(String[] args)
+    {
+        Options options = null;
+        int rf = 0;
+        int tokens = 0;
+        int nodes = 0;
+        IPartitioner partitioner = null;
+        int[] racksDef = null;
+
+        OutputHandler logger = null;
+
+        try
+        {
+            // disable the summary statistics logging, since this is a 
command-line tool with dedicated output
+            ((Logger) 
LoggerFactory.getLogger(TokenAllocation.class)).setLevel(Level.ERROR);
+
+            Util.initDatabaseDescriptor();
+            options = getOptions();
+            CommandLine cmd = parseCommandLine(args, options);
+
+            rf = Integer.parseInt(cmd.getOptionValue(RF));
+            tokens = Integer.parseInt(cmd.getOptionValue(TOKENS));
+            nodes = Integer.parseInt(cmd.getOptionValue(NODES));
+            logger = new OutputHandler.SystemOutput(cmd.hasOption(VERBOSE), 
true, true);
+
+            partitioner = 
FBUtilities.newPartitioner(cmd.getOptionValue(PARTITIONER, 
Murmur3Partitioner.class.getSimpleName()));
+            racksDef = getRacks(cmd.getOptionValue(RACKS, 
cmd.getOptionValue(NODES)));
+            if (Arrays.stream(racksDef).sum() != nodes)
+            {
+                throw new AssertionError(String.format("The sum of nodes in 
each rack %s must equal total node count %s.",
+                                                       
cmd.getOptionValue(RACKS),
+                                                       
cmd.getOptionValue(NODES)));
+            }
+        }
+        catch (NumberFormatException e)
+        {
+            System.err.println("Invalid integer " + e.getMessage());
+            System.out.println();
+            printUsage(options);
+            System.exit(1);
+        }
+        catch (AssertionError | ConfigurationException | ParseException t)
+        {
+            System.err.println(t.getMessage());
+            System.out.println();
+            printUsage(options);
+            System.exit(1);
+        }
+
+        try
+        {
+            logger.output(String.format("Generating tokens for %d nodes with 
%d vnodes each for replication factor %d and partitioner %s",
+                                             nodes, tokens, rf, 
partitioner.getClass().getSimpleName()));
+
+            for (OfflineTokenAllocator.FakeNode node : 
OfflineTokenAllocator.allocate(rf, tokens, racksDef, logger, partitioner))
+                logger.output(String.format("Node %d rack %d: %s", 
node.nodeId(), node.rackId(), node.tokens().toString()));
+        }
+        catch (Throwable t)
+        {
+            logger.warn("Error running tool.", t);
+            System.exit(1);
+        }
+    }
+
+    private static int[] getRacks(String racksDef)
+    {
+        return 
Arrays.stream(racksDef.split(",")).mapToInt(Integer::parseInt).toArray();
+    }
+
+    private static CommandLine parseCommandLine(String[] args, Options 
options) throws ParseException
+    {
+        return new GnuParser().parse(options, args, false);
+    }
+
+    private static Options getOptions()
+    {
+        Options options = new Options();
+        options.addOption(requiredOption("n", NODES, true, "Number of 
nodes."));
+        options.addOption(requiredOption("t", TOKENS, true, "Number of 
tokens/vnodes per node."));
+        options.addOption(requiredOption(null, RF, true, "Replication 
factor."));
+        options.addOption("p", PARTITIONER, true, "Database partitioner, 
either Murmur3Partitioner or RandomPartitioner.");
+        options.addOption(null, RACKS, true,
+                          "Number of nodes per rack, separated by commas. Must 
add up to the total node count.\n" +
+                          "For example, 'generatetokens -n 30 -t 8 --rf 3 
--racks 10,10,10' will generate tokens for\n" +
+                          "three racks of 10 nodes each.");
+        options.addOption("v", VERBOSE, false, "Verbose logging.");
+        return options;
+    }
+
+    private static Option requiredOption(String shortOpt, String longOpt, 
boolean hasArg, String description)
+    {
+        Option option = new Option(shortOpt, longOpt, hasArg, description);
+        option.setRequired(true);
+        return option;
+    }
+
+    public static void printUsage(Options options)
+    {
+        String usage = "generatetokens -n NODES -t TOKENS --rf 
REPLICATION_FACTOR";
+        String header = "--\n" +
+                        "Generates tokens for a datacenter with the given 
number of nodes using the token allocation algorithm.\n" +
+                        "Options are:";
+        new HelpFormatter().printHelp(usage, header, options, "");
+    }
+}
+
diff --git a/src/java/org/apache/cassandra/utils/OutputHandler.java 
b/src/java/org/apache/cassandra/utils/OutputHandler.java
index 88c54ce..3b10d61 100644
--- a/src/java/org/apache/cassandra/utils/OutputHandler.java
+++ b/src/java/org/apache/cassandra/utils/OutputHandler.java
@@ -18,6 +18,8 @@
  */
 package org.apache.cassandra.utils;
 
+import java.io.PrintStream;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,11 +64,18 @@ public interface OutputHandler
     {
         public final boolean debug;
         public final boolean printStack;
+        public final PrintStream warnOut;
 
         public SystemOutput(boolean debug, boolean printStack)
         {
+            this(debug, printStack, false);
+        }
+
+        public SystemOutput(boolean debug, boolean printStack, boolean 
logWarnToStdErr)
+        {
             this.debug = debug;
             this.printStack = printStack;
+            this.warnOut = logWarnToStdErr ? System.err : System.out;
         }
 
         public void output(String msg)
@@ -87,9 +96,9 @@ public interface OutputHandler
 
         public void warn(String msg, Throwable th)
         {
-            System.out.println("WARNING: " + msg);
+            warnOut.println("WARNING: " + msg);
             if (printStack && th != null)
-                th.printStackTrace(System.out);
+                th.printStackTrace(warnOut);
         }
     }
 }
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java 
b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index c5cce58..44ea5a9 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -17,27 +17,14 @@
  */
 package org.apache.cassandra.dht;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-import org.junit.Assert;
-
-import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 import java.util.Random;
-import java.util.UUID;
 
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
-
-import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
-import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -46,21 +33,21 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.locator.Replica;
-import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.tokenallocator.TokenAllocation;
+import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.IFailureDetector;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.locator.RackInferringSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.StreamOperation;
-import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
 
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class BootStrapperTest
@@ -164,191 +151,9 @@ public class BootStrapperTest
             List<Token> tokens = Lists.newArrayListWithCapacity(numVNodes);
             for (int j = 0; j < numVNodes; ++j)
                 tokens.add(p.getRandomToken(rand));
-            
-            tmd.updateNormalTokens(tokens, addr);
-        }
-    }
-    
-    @Test
-    public void testAllocateTokens() throws UnknownHostException
-    {
-        int vn = 16;
-        String ks = "BootStrapperTestKeyspace3";
-        TokenMetadata tm = new TokenMetadata();
-        generateFakeEndpoints(tm, 10, vn);
-        InetAddressAndPort addr = FBUtilities.getBroadcastAddressAndPort();
-        allocateTokensForNode(vn, ks, tm, addr);
-    }
-
-    @Test
-    public void testAllocateTokensLocalRf() throws UnknownHostException
-    {
-        int vn = 16;
-        int allocateTokensForLocalRf = 3;
-        TokenMetadata tm = new TokenMetadata();
-        generateFakeEndpoints(tm, 10, vn);
-        InetAddressAndPort addr = FBUtilities.getBroadcastAddressAndPort();
-        allocateTokensForNode(vn, allocateTokensForLocalRf, tm, addr);
-    }
-
-    public void testAllocateTokensNetworkStrategy(int rackCount, int replicas) 
throws UnknownHostException
-    {
-        IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch();
-        try
-        {
-            DatabaseDescriptor.setEndpointSnitch(new RackInferringSnitch());
-            int vn = 16;
-            String ks = "BootStrapperTestNTSKeyspace" + rackCount + replicas;
-            String dc = "1";
-
-            // Register peers with expected DC for NetworkTopologyStrategy.
-            TokenMetadata metadata = 
StorageService.instance.getTokenMetadata();
-            metadata.clearUnsafe();
-            metadata.updateHostId(UUID.randomUUID(), 
InetAddressAndPort.getByName("127.1.0.99"));
-            metadata.updateHostId(UUID.randomUUID(), 
InetAddressAndPort.getByName("127.15.0.99"));
-
-            SchemaLoader.createKeyspace(ks, KeyspaceParams.nts(dc, replicas, 
"15", 15), SchemaLoader.standardCFMD(ks, "Standard1"));
-            TokenMetadata tm = StorageService.instance.getTokenMetadata();
-            tm.clearUnsafe();
-            for (int i = 0; i < rackCount; ++i)
-                generateFakeEndpoints(tm, 10, vn, dc, Integer.toString(i));
-            InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc 
+ ".0.99");
-            allocateTokensForNode(vn, ks, tm, addr);
-            // Note: Not matching replication factor in second datacentre, but 
this should not affect us.
-        } finally {
-            DatabaseDescriptor.setEndpointSnitch(oldSnitch);
-        }
-    }
-
-    @Test
-    public void testAllocateTokensNetworkStrategyOneRack() throws 
UnknownHostException
-    {
-        testAllocateTokensNetworkStrategy(1, 3);
-    }
-
-    @Test(expected = ConfigurationException.class)
-    public void testAllocateTokensNetworkStrategyTwoRacks() throws 
UnknownHostException
-    {
-        testAllocateTokensNetworkStrategy(2, 3);
-    }
 
-    @Test
-    public void testAllocateTokensNetworkStrategyThreeRacks() throws 
UnknownHostException
-    {
-        testAllocateTokensNetworkStrategy(3, 3);
-    }
-
-    @Test
-    public void testAllocateTokensNetworkStrategyFiveRacks() throws 
UnknownHostException
-    {
-        testAllocateTokensNetworkStrategy(5, 3);
-    }
-
-    @Test
-    public void testAllocateTokensNetworkStrategyOneRackOneReplica() throws 
UnknownHostException
-    {
-        testAllocateTokensNetworkStrategy(1, 1);
-    }
-
-    private void allocateTokensForNode(int vn, String ks, TokenMetadata tm, 
InetAddressAndPort addr)
-    {
-        SummaryStatistics os = 
TokenAllocation.replicatedOwnershipStats(tm.cloneOnlyTokenMap(), 
Keyspace.open(ks).getReplicationStrategy(), addr);
-        Collection<Token> tokens = BootStrapper.allocateTokens(tm, addr, ks, 
vn, 0);
-        assertEquals(vn, tokens.size());
-        tm.updateNormalTokens(tokens, addr);
-        SummaryStatistics ns = 
TokenAllocation.replicatedOwnershipStats(tm.cloneOnlyTokenMap(), 
Keyspace.open(ks).getReplicationStrategy(), addr);
-        verifyImprovement(os, ns);
-    }
-
-    private void allocateTokensForNode(int vn, int rf, TokenMetadata tm, 
InetAddressAndPort addr)
-    {
-        Collection<Token> tokens = BootStrapper.allocateTokens(tm, addr, rf, 
vn, 0);
-        assertEquals(vn, tokens.size());
-        tm.updateNormalTokens(tokens, addr);
-        // SummaryStatistics is not implemented for 
`allocate_tokens_for_local_replication_factor` so can't be verified
-    }
-
-    private void verifyImprovement(SummaryStatistics os, SummaryStatistics ns)
-    {
-        if (ns.getStandardDeviation() > os.getStandardDeviation())
-        {
-            fail(String.format("Token allocation unexpectedly increased 
standard deviation.\nStats before:\n%s\nStats after:\n%s", os, ns));
-        }
-    }
-
-    @Test
-    public void testAllocateTokensRfEqRacks() throws UnknownHostException
-    {
-        IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch();
-        try
-        {
-            DatabaseDescriptor.setEndpointSnitch(new RackInferringSnitch());
-            int vn = 8;
-            int replicas = 3;
-            int rackCount = replicas;
-            String ks = "BootStrapperTestNTSKeyspaceRfEqRacks";
-            String dc = "1";
-
-            TokenMetadata metadata = 
StorageService.instance.getTokenMetadata();
-            metadata.clearUnsafe();
-            metadata.updateHostId(UUID.randomUUID(), 
InetAddressAndPort.getByName("127.1.0.99"));
-            metadata.updateHostId(UUID.randomUUID(), 
InetAddressAndPort.getByName("127.15.0.99"));
-
-            SchemaLoader.createKeyspace(ks, KeyspaceParams.nts(dc, replicas, 
"15", 15), SchemaLoader.standardCFMD(ks, "Standard1"));
-            int base = 5;
-            for (int i = 0; i < rackCount; ++i)
-                generateFakeEndpoints(metadata, base << i, vn, dc, 
Integer.toString(i));     // unbalanced racks
-
-            int cnt = 5;
-            for (int i = 0; i < cnt; ++i)
-                allocateTokensForNode(vn, ks, metadata, 
InetAddressAndPort.getByName("127." + dc + ".0." + (99 + i)));
-
-            double target = 1.0 / (base + cnt);
-            double permittedOver = 1.0 / (2 * vn + 1) + 0.01;
-
-            Map<InetAddress, Float> ownership = 
StorageService.instance.effectiveOwnership(ks);
-            boolean failed = false;
-            for (Map.Entry<InetAddress, Float> o : ownership.entrySet())
-            {
-                int rack = o.getKey().getAddress()[2];
-                if (rack != 0)
-                    continue;
-
-                System.out.format("Node %s owns %f ratio to optimal %.2f\n", 
o.getKey(), o.getValue(), o.getValue() / target);
-                if (o.getValue()/target > 1 + permittedOver)
-                    failed = true;
-            }
-            Assert.assertFalse(String.format("One of the nodes in the rack has 
over %.2f%% overutilization.", permittedOver * 100), failed);
-        } finally {
-            DatabaseDescriptor.setEndpointSnitch(oldSnitch);
+            tmd.updateNormalTokens(tokens, addr);
         }
     }
 
-    @Test
-    public void testAllocateTokensMultipleKeyspaces() throws 
UnknownHostException
-    {
-        // TODO: This scenario isn't supported very well. Investigate a 
multi-keyspace version of the algorithm.
-        int vn = 16;
-        String ks3 = "BootStrapperTestKeyspace4"; // RF = 3
-        String ks2 = "BootStrapperTestKeyspace5"; // RF = 2
-
-        TokenMetadata tm = new TokenMetadata();
-        generateFakeEndpoints(tm, 10, vn);
-        
-        InetAddressAndPort dcaddr = FBUtilities.getBroadcastAddressAndPort();
-        SummaryStatistics os3 = TokenAllocation.replicatedOwnershipStats(tm, 
Keyspace.open(ks3).getReplicationStrategy(), dcaddr);
-        SummaryStatistics os2 = TokenAllocation.replicatedOwnershipStats(tm, 
Keyspace.open(ks2).getReplicationStrategy(), dcaddr);
-        String cks = ks3;
-        String nks = ks2;
-        for (int i=11; i<=20; ++i)
-        {
-            allocateTokensForNode(vn, cks, tm, 
InetAddressAndPort.getByName("127.0.0." + (i + 1)));
-            String t = cks; cks = nks; nks = t;
-        }
-        
-        SummaryStatistics ns3 = TokenAllocation.replicatedOwnershipStats(tm, 
Keyspace.open(ks3).getReplicationStrategy(), dcaddr);
-        SummaryStatistics ns2 = TokenAllocation.replicatedOwnershipStats(tm, 
Keyspace.open(ks2).getReplicationStrategy(), dcaddr);
-        verifyImprovement(os3, ns3);
-        verifyImprovement(os2, ns2);
-    }
 }
diff --git 
a/test/unit/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocatorTest.java
 
b/test/unit/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocatorTest.java
new file mode 100644
index 0000000..832d13d
--- /dev/null
+++ 
b/test/unit/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocatorTest.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.dht.tokenallocator;
+
+import java.util.Collection;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+import org.assertj.core.api.Assertions;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.tools.Util;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static 
org.apache.cassandra.dht.tokenallocator.OfflineTokenAllocator.allocate;
+
+public class OfflineTokenAllocatorTest
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(OfflineTokenAllocatorTest.class);
+    private static final OutputHandler FAIL_ON_WARN_OUTPUT = new 
SystemOutputImpl();
+
+    @Before
+    public void setup()
+    {
+        Util.initDatabaseDescriptor();
+    }
+
+    @Test(expected = UnsupportedOperationException.class)
+    public void testUnsupportedPartitioner()
+    {
+        List<OfflineTokenAllocator.FakeNode> nodes = allocate(3, 4, new 
int[]{1,1,1}, FAIL_ON_WARN_OUTPUT, ByteOrderedPartitioner.instance);
+        Assert.assertEquals(3, nodes.size());
+    }
+
+    /**
+     * Cycle through a matrix of valid ranges.
+     */
+    @Test
+    public void testTokenGenerations()
+    {
+        for (int numTokens = 1; numTokens <= 16 ; ++numTokens)
+        {
+            for (int rf = 1; rf <=5; ++rf)
+            {
+                int nodeCount = 32;
+                for (int racks = 1; racks <= 10; ++racks)
+                {
+                    int[] nodeToRack = makeRackCountArray(nodeCount, racks);
+                    for (IPartitioner partitioner : new IPartitioner[] { 
Murmur3Partitioner.instance, RandomPartitioner.instance })
+                    {
+                        logger.info("Testing offline token allocator for 
numTokens={}, rf={}, racks={}, nodeToRack={}, partitioner={}",
+                                    numTokens, rf, racks, nodeToRack, 
partitioner);
+                        assertTokensAndNodeCount(numTokens, nodeCount, 
allocate(rf,
+                                                                               
 numTokens,
+                                                                               
 nodeToRack,
+                                                                               
 new SystemOutputImpl(rf, racks),
+                                                                               
 partitioner));
+                    }
+                }
+            }
+        }
+    }
+
+    private void assertTokensAndNodeCount(int numTokens, int nodeCount, 
List<OfflineTokenAllocator.FakeNode> nodes)
+    {
+        assertEquals(nodeCount, nodes.size());
+        Collection<Token> allTokens = Lists.newArrayList();
+        for (OfflineTokenAllocator.FakeNode node : nodes)
+        {
+            Assertions.assertThat(node.tokens()).hasSize(numTokens);
+            
Assertions.assertThat(allTokens).doesNotContainAnyElementsOf(node.tokens());
+            allTokens.addAll(node.tokens());
+        }
+    }
+
+    private static int[] makeRackCountArray(int nodes, int racks)
+    {
+        assert nodes > 0;
+        assert racks > 0;
+        // Distribute nodes among the racks in round-robin fashion in the 
order the user is supposed to start them.
+        int[] rackCounts = new int[racks];
+        int rack = 0;
+        for (int node = 0; node < nodes; node++)
+        {
+            rackCounts[rack]++;
+            if (++rack == racks)
+                rack = 0;
+        }
+        return rackCounts;
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testTokenGenerator_more_rf_than_racks()
+    {
+        OfflineTokenAllocator.allocate(3, 16, new int[]{1, 1}, 
FAIL_ON_WARN_OUTPUT, Murmur3Partitioner.instance);
+    }
+
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testTokenGenerator_more_rf_than_nodes()
+    {
+        OfflineTokenAllocator.allocate(3, 16, new int[]{2}, 
FAIL_ON_WARN_OUTPUT, Murmur3Partitioner.instance);
+    }
+
+    @Test
+    public void testTokenGenerator_single_rack_or_single_rf()
+    {
+        int numTokens = 16;
+        // Simple cases, single rack or single replication.
+        assertTokensAndNodeCount(numTokens, 1, allocate(1,
+                                                        numTokens,
+                                                        new int[]{1},
+                                                        FAIL_ON_WARN_OUTPUT,
+                                                        
Murmur3Partitioner.instance));
+        assertTokensAndNodeCount(numTokens, 2, allocate(1,
+                                                        numTokens,
+                                                        new int[]{1, 1},
+                                                        FAIL_ON_WARN_OUTPUT,
+                                                        
Murmur3Partitioner.instance));
+        assertTokensAndNodeCount(numTokens, 2, allocate(1,
+                                                        numTokens,
+                                                        new int[]{2},
+                                                        FAIL_ON_WARN_OUTPUT,
+                                                        
Murmur3Partitioner.instance));
+    }
+
+    @Test
+    public void testTokenGenerator_unbalanced_racks()
+    {
+        int numTokens = 16;
+        // Simple cases, single rack or single replication.
+        assertTokensAndNodeCount(numTokens, 6, allocate(1,
+                                                        numTokens,
+                                                        new int[]{5, 1},
+                                                        FAIL_ON_WARN_OUTPUT,
+                                                        
Murmur3Partitioner.instance));
+        assertTokensAndNodeCount(numTokens, 7, allocate(1,
+                                                        numTokens,
+                                                        new int[]{5, 1, 1},
+                                                        FAIL_ON_WARN_OUTPUT,
+                                                        
Murmur3Partitioner.instance));
+        assertTokensAndNodeCount(numTokens, 6, allocate(3,
+                                                        numTokens,
+                                                        new int[]{5, 1},
+                                                        FAIL_ON_WARN_OUTPUT,
+                                                        
Murmur3Partitioner.instance));
+        assertTokensAndNodeCount(numTokens, 7, allocate(3,
+                                                        numTokens,
+                                                        new int[]{5, 1, 1},
+                                                        FAIL_ON_WARN_OUTPUT,
+                                                        
Murmur3Partitioner.instance));
+    }
+
+    private static class SystemOutputImpl extends OutputHandler.SystemOutput
+    {
+        private final int rf;
+        private final int racks;
+
+        private SystemOutputImpl()
+        {
+            super(true, true);
+            rf = racks = 1;
+        }
+
+        private SystemOutputImpl(int rf, int racks)
+        {
+            super(true, true);
+            this.rf = rf;
+            this.racks = racks;
+        }
+
+        @Override
+        public void warn(String msg)
+        {
+            // We can only guarantee that ownership stdev won't increase above 
the warn threshold for racks==1 or racks==rf
+            if (racks == 1 || racks == rf)
+                fail(msg);
+            else
+                super.warn(msg);
+        }
+
+        @Override
+        public void warn(String msg, Throwable th)
+        {
+            // We can only guarantee that ownership stdev won't increase above 
the warn threshold for racks==1 or racks==rf
+            if (racks == 1 || racks == rf)
+                fail(msg);
+            else
+                super.warn(msg, th);
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java 
b/test/unit/org/apache/cassandra/dht/tokenallocator/TokenAllocationTest.java
similarity index 53%
copy from test/unit/org/apache/cassandra/dht/BootStrapperTest.java
copy to 
test/unit/org/apache/cassandra/dht/tokenallocator/TokenAllocationTest.java
index c5cce58..ac372e4 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/tokenallocator/TokenAllocationTest.java
@@ -15,12 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.cassandra.dht;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-import org.junit.Assert;
+package org.apache.cassandra.dht.tokenallocator;
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -30,44 +26,37 @@ import java.util.Map;
 import java.util.Random;
 import java.util.UUID;
 
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-
-import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
 import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
-
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.locator.Replica;
-import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.tokenallocator.TokenAllocation;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.gms.IFailureDetectionEventListener;
-import org.apache.cassandra.gms.IFailureDetector;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.RackInferringSnitch;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.utils.FBUtilities;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
-public class BootStrapperTest
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class TokenAllocationTest
 {
     static IPartitioner oldPartitioner;
+    static Random rand = new Random(1);
 
-    static Predicate<Replica> originalAlivePredicate = 
RangeStreamer.ALIVE_PREDICATE;
     @BeforeClass
     public static void setup() throws ConfigurationException
     {
@@ -75,149 +64,66 @@ public class BootStrapperTest
         oldPartitioner = 
StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
         SchemaLoader.startGossiper();
         SchemaLoader.prepareServer();
-        SchemaLoader.schemaDefinition("BootStrapperTest");
-        RangeStreamer.ALIVE_PREDICATE = Predicates.alwaysTrue();
+        SchemaLoader.schemaDefinition("TokenAllocationTest");
     }
 
     @AfterClass
     public static void tearDown()
     {
         DatabaseDescriptor.setPartitionerUnsafe(oldPartitioner);
-        RangeStreamer.ALIVE_PREDICATE = originalAlivePredicate;
     }
 
-    @Test
-    public void testSourceTargetComputation() throws UnknownHostException
+    private static TokenAllocation createForTest(TokenMetadata tokenMetadata, 
int replicas, int numTokens)
     {
-        final int[] clusterSizes = new int[] { 1, 3, 5, 10, 100};
-        for (String keyspaceName : 
Schema.instance.getNonLocalStrategyKeyspaces())
-        {
-            int replicationFactor = 
Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor().allReplicas;
-            for (int clusterSize : clusterSizes)
-                if (clusterSize >= replicationFactor)
-                    testSourceTargetComputation(keyspaceName, clusterSize, 
replicationFactor);
-        }
-    }
-
-    private RangeStreamer testSourceTargetComputation(String keyspaceName, int 
numOldNodes, int replicationFactor) throws UnknownHostException
-    {
-        StorageService ss = StorageService.instance;
-        TokenMetadata tmd = ss.getTokenMetadata();
-
-        generateFakeEndpoints(numOldNodes);
-        Token myToken = tmd.partitioner.getRandomToken();
-        InetAddressAndPort myEndpoint = 
InetAddressAndPort.getByName("127.0.0.1");
-
-        assertEquals(numOldNodes, tmd.sortedTokens().size());
-        IFailureDetector mockFailureDetector = new IFailureDetector()
-        {
-            public boolean isAlive(InetAddressAndPort ep)
-            {
-                return true;
-            }
-
-            public void interpret(InetAddressAndPort ep) { throw new 
UnsupportedOperationException(); }
-            public void report(InetAddressAndPort ep) { throw new 
UnsupportedOperationException(); }
-            public void 
registerFailureDetectionEventListener(IFailureDetectionEventListener listener) 
{ throw new UnsupportedOperationException(); }
-            public void 
unregisterFailureDetectionEventListener(IFailureDetectionEventListener 
listener) { throw new UnsupportedOperationException(); }
-            public void remove(InetAddressAndPort ep) { throw new 
UnsupportedOperationException(); }
-            public void forceConviction(InetAddressAndPort ep) { throw new 
UnsupportedOperationException(); }
-        };
-        RangeStreamer s = new RangeStreamer(tmd, null, myEndpoint, 
StreamOperation.BOOTSTRAP, true, DatabaseDescriptor.getEndpointSnitch(), new 
StreamStateStore(), mockFailureDetector, false, 1);
-        assertNotNull(Keyspace.open(keyspaceName));
-        s.addRanges(keyspaceName, 
Keyspace.open(keyspaceName).getReplicationStrategy().getPendingAddressRanges(tmd,
 myToken, myEndpoint));
-
-
-        Multimap<InetAddressAndPort, FetchReplica> toFetch = 
s.toFetch().get(keyspaceName);
-
-        // Check we get get RF new ranges in total
-        assertEquals(replicationFactor, toFetch.size());
-
-        // there isn't any point in testing the size of these collections for 
any specific size.  When a random partitioner
-        // is used, they will vary.
-        assert toFetch.values().size() > 0;
-        assert toFetch.keys().stream().noneMatch(myEndpoint::equals);
-        return s;
-    }
-
-    private void generateFakeEndpoints(int numOldNodes) throws 
UnknownHostException
-    {
-        generateFakeEndpoints(StorageService.instance.getTokenMetadata(), 
numOldNodes, 1);
-    }
-
-    private void generateFakeEndpoints(TokenMetadata tmd, int numOldNodes, int 
numVNodes) throws UnknownHostException
-    {
-        tmd.clearUnsafe();
-        generateFakeEndpoints(tmd, numOldNodes, numVNodes, "0", "0");
+        return TokenAllocation.create(DatabaseDescriptor.getEndpointSnitch(), 
tokenMetadata, replicas, numTokens);
     }
 
-    Random rand = new Random(1);
-
-    private void generateFakeEndpoints(TokenMetadata tmd, int numOldNodes, int 
numVNodes, String dc, String rack) throws UnknownHostException
-    {
-        IPartitioner p = tmd.partitioner;
-
-        for (int i = 1; i <= numOldNodes; i++)
-        {
-            // leave .1 for myEndpoint
-            InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc 
+ "." + rack + "." + (i + 1));
-            List<Token> tokens = Lists.newArrayListWithCapacity(numVNodes);
-            for (int j = 0; j < numVNodes; ++j)
-                tokens.add(p.getRandomToken(rand));
-            
-            tmd.updateNormalTokens(tokens, addr);
-        }
-    }
-    
     @Test
-    public void testAllocateTokens() throws UnknownHostException
+    public void testAllocateTokensForKeyspace() throws UnknownHostException
     {
         int vn = 16;
-        String ks = "BootStrapperTestKeyspace3";
+        String ks = "TokenAllocationTestKeyspace3";
         TokenMetadata tm = new TokenMetadata();
         generateFakeEndpoints(tm, 10, vn);
         InetAddressAndPort addr = FBUtilities.getBroadcastAddressAndPort();
-        allocateTokensForNode(vn, ks, tm, addr);
+        allocateTokensForKeyspace(vn, ks, tm, addr);
     }
 
     @Test
-    public void testAllocateTokensLocalRf() throws UnknownHostException
+    public void testAllocateTokensForLocalRF() throws UnknownHostException
     {
         int vn = 16;
         int allocateTokensForLocalRf = 3;
         TokenMetadata tm = new TokenMetadata();
         generateFakeEndpoints(tm, 10, vn);
         InetAddressAndPort addr = FBUtilities.getBroadcastAddressAndPort();
-        allocateTokensForNode(vn, allocateTokensForLocalRf, tm, addr);
+        allocateTokensForLocalReplicationFactor(vn, allocateTokensForLocalRf, 
tm, addr);
     }
 
-    public void testAllocateTokensNetworkStrategy(int rackCount, int replicas) 
throws UnknownHostException
+    private Collection<Token> allocateTokensForKeyspace(int vnodes, String 
keyspace, TokenMetadata tm, InetAddressAndPort addr)
     {
-        IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch();
-        try
-        {
-            DatabaseDescriptor.setEndpointSnitch(new RackInferringSnitch());
-            int vn = 16;
-            String ks = "BootStrapperTestNTSKeyspace" + rackCount + replicas;
-            String dc = "1";
+        AbstractReplicationStrategy rs = 
Keyspace.open(keyspace).getReplicationStrategy();
+        TokenAllocation tokenAllocation = TokenAllocation.create(tm, rs, 
vnodes);
+        return allocateAndVerify(vnodes, addr, tokenAllocation);
+    }
 
-            // Register peers with expected DC for NetworkTopologyStrategy.
-            TokenMetadata metadata = 
StorageService.instance.getTokenMetadata();
-            metadata.clearUnsafe();
-            metadata.updateHostId(UUID.randomUUID(), 
InetAddressAndPort.getByName("127.1.0.99"));
-            metadata.updateHostId(UUID.randomUUID(), 
InetAddressAndPort.getByName("127.15.0.99"));
+    private void allocateTokensForLocalReplicationFactor(int vnodes, int rf, 
TokenMetadata tm, InetAddressAndPort addr)
+    {
+        TokenAllocation tokenAllocation = createForTest(tm, rf, vnodes);
+        allocateAndVerify(vnodes, addr, tokenAllocation);
+    }
 
-            SchemaLoader.createKeyspace(ks, KeyspaceParams.nts(dc, replicas, 
"15", 15), SchemaLoader.standardCFMD(ks, "Standard1"));
-            TokenMetadata tm = StorageService.instance.getTokenMetadata();
-            tm.clearUnsafe();
-            for (int i = 0; i < rackCount; ++i)
-                generateFakeEndpoints(tm, 10, vn, dc, Integer.toString(i));
-            InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc 
+ ".0.99");
-            allocateTokensForNode(vn, ks, tm, addr);
-            // Note: Not matching replication factor in second datacentre, but 
this should not affect us.
-        } finally {
-            DatabaseDescriptor.setEndpointSnitch(oldSnitch);
+    private Collection<Token> allocateAndVerify(int vnodes, InetAddressAndPort 
addr, TokenAllocation tokenAllocation)
+    {
+        SummaryStatistics os = 
tokenAllocation.getAllocationRingOwnership(addr);
+        Collection<Token> tokens = tokenAllocation.allocate(addr);
+        assertEquals(vnodes, tokens.size());
+        SummaryStatistics ns = 
tokenAllocation.getAllocationRingOwnership(addr);
+        if (ns.getStandardDeviation() > os.getStandardDeviation())
+        {
+            fail(String.format("Token allocation unexpectedly increased 
standard deviation.\nStats before:\n%s\nStats after:\n%s", os, ns));
         }
+        return tokens;
     }
 
     @Test
@@ -250,29 +156,33 @@ public class BootStrapperTest
         testAllocateTokensNetworkStrategy(1, 1);
     }
 
-    private void allocateTokensForNode(int vn, String ks, TokenMetadata tm, 
InetAddressAndPort addr)
-    {
-        SummaryStatistics os = 
TokenAllocation.replicatedOwnershipStats(tm.cloneOnlyTokenMap(), 
Keyspace.open(ks).getReplicationStrategy(), addr);
-        Collection<Token> tokens = BootStrapper.allocateTokens(tm, addr, ks, 
vn, 0);
-        assertEquals(vn, tokens.size());
-        tm.updateNormalTokens(tokens, addr);
-        SummaryStatistics ns = 
TokenAllocation.replicatedOwnershipStats(tm.cloneOnlyTokenMap(), 
Keyspace.open(ks).getReplicationStrategy(), addr);
-        verifyImprovement(os, ns);
-    }
-
-    private void allocateTokensForNode(int vn, int rf, TokenMetadata tm, 
InetAddressAndPort addr)
-    {
-        Collection<Token> tokens = BootStrapper.allocateTokens(tm, addr, rf, 
vn, 0);
-        assertEquals(vn, tokens.size());
-        tm.updateNormalTokens(tokens, addr);
-        // SummaryStatistics is not implemented for 
`allocate_tokens_for_local_replication_factor` so can't be verified
-    }
 
-    private void verifyImprovement(SummaryStatistics os, SummaryStatistics ns)
+    public void testAllocateTokensNetworkStrategy(int rackCount, int replicas) 
throws UnknownHostException
     {
-        if (ns.getStandardDeviation() > os.getStandardDeviation())
+        IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch();
+        try
         {
-            fail(String.format("Token allocation unexpectedly increased 
standard deviation.\nStats before:\n%s\nStats after:\n%s", os, ns));
+            DatabaseDescriptor.setEndpointSnitch(new RackInferringSnitch());
+            int vn = 16;
+            String ks = "TokenAllocationTestNTSKeyspace" + rackCount + 
replicas;
+            String dc = "1";
+
+            // Register peers with expected DC for NetworkTopologyStrategy.
+            TokenMetadata metadata = 
StorageService.instance.getTokenMetadata();
+            metadata.clearUnsafe();
+            metadata.updateHostId(UUID.randomUUID(), 
InetAddressAndPort.getByName("127.1.0.99"));
+            metadata.updateHostId(UUID.randomUUID(), 
InetAddressAndPort.getByName("127.15.0.99"));
+
+            SchemaLoader.createKeyspace(ks, KeyspaceParams.nts(dc, replicas, 
"15", 15), SchemaLoader.standardCFMD(ks, "Standard1"));
+            TokenMetadata tm = StorageService.instance.getTokenMetadata();
+            tm.clearUnsafe();
+            for (int i = 0; i < rackCount; ++i)
+                generateFakeEndpoints(tm, 10, vn, dc, Integer.toString(i));
+            InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc 
+ ".0.99");
+            allocateTokensForKeyspace(vn, ks, tm, addr);
+            // Note: Not matching replication factor in second datacentre, but 
this should not affect us.
+        } finally {
+            DatabaseDescriptor.setEndpointSnitch(oldSnitch);
         }
     }
 
@@ -286,7 +196,7 @@ public class BootStrapperTest
             int vn = 8;
             int replicas = 3;
             int rackCount = replicas;
-            String ks = "BootStrapperTestNTSKeyspaceRfEqRacks";
+            String ks = "TokenAllocationTestNTSKeyspaceRfEqRacks";
             String dc = "1";
 
             TokenMetadata metadata = 
StorageService.instance.getTokenMetadata();
@@ -301,7 +211,11 @@ public class BootStrapperTest
 
             int cnt = 5;
             for (int i = 0; i < cnt; ++i)
-                allocateTokensForNode(vn, ks, metadata, 
InetAddressAndPort.getByName("127." + dc + ".0." + (99 + i)));
+            {
+                InetAddressAndPort endpoint = 
InetAddressAndPort.getByName("127." + dc + ".0." + (99 + i));
+                Collection<Token> tokens = allocateTokensForKeyspace(vn, ks, 
metadata, endpoint);
+                metadata.updateNormalTokens(tokens, endpoint);
+            }
 
             double target = 1.0 / (base + cnt);
             double permittedOver = 1.0 / (2 * vn + 1) + 0.01;
@@ -324,31 +238,73 @@ public class BootStrapperTest
         }
     }
 
+
+    /**
+     * TODO: This scenario isn't supported very well. Investigate a 
multi-keyspace version of the algorithm.
+     */
     @Test
     public void testAllocateTokensMultipleKeyspaces() throws 
UnknownHostException
     {
-        // TODO: This scenario isn't supported very well. Investigate a 
multi-keyspace version of the algorithm.
-        int vn = 16;
-        String ks3 = "BootStrapperTestKeyspace4"; // RF = 3
-        String ks2 = "BootStrapperTestKeyspace5"; // RF = 2
+        final int TOKENS = 16;
+
+        TokenMetadata tokenMetadata = new TokenMetadata();
+        generateFakeEndpoints(tokenMetadata, 10, TOKENS);
+
+        // Do not clone token metadata so tokens allocated by different 
allocators are reflected on the parent TokenMetadata
+        TokenAllocation rf2Allocator = createForTest(tokenMetadata, 2, TOKENS);
+        TokenAllocation rf3Allocator = createForTest(tokenMetadata, 3, TOKENS);
+
+        SummaryStatistics rf2StatsBefore = 
rf2Allocator.getAllocationRingOwnership(FBUtilities.getBroadcastAddressAndPort());
+        SummaryStatistics rf3StatsBefore = 
rf3Allocator.getAllocationRingOwnership(FBUtilities.getBroadcastAddressAndPort());
+
+        TokenAllocation current = rf3Allocator;
+        TokenAllocation next = rf2Allocator;
 
-        TokenMetadata tm = new TokenMetadata();
-        generateFakeEndpoints(tm, 10, vn);
-        
-        InetAddressAndPort dcaddr = FBUtilities.getBroadcastAddressAndPort();
-        SummaryStatistics os3 = TokenAllocation.replicatedOwnershipStats(tm, 
Keyspace.open(ks3).getReplicationStrategy(), dcaddr);
-        SummaryStatistics os2 = TokenAllocation.replicatedOwnershipStats(tm, 
Keyspace.open(ks2).getReplicationStrategy(), dcaddr);
-        String cks = ks3;
-        String nks = ks2;
         for (int i=11; i<=20; ++i)
         {
-            allocateTokensForNode(vn, cks, tm, 
InetAddressAndPort.getByName("127.0.0." + (i + 1)));
-            String t = cks; cks = nks; nks = t;
+            InetAddressAndPort endpoint = 
InetAddressAndPort.getByName("127.0.0." + (i + 1));
+            Collection<Token> tokens = current.allocate(endpoint);
+            // Update tokens on next to verify ownership calculation below
+            next.tokenMetadata.updateNormalTokens(tokens, endpoint);
+            TokenAllocation tmp = current;
+            current = next;
+            next = tmp;
+        }
+
+        SummaryStatistics rf2StatsAfter = 
rf2Allocator.getAllocationRingOwnership(FBUtilities.getBroadcastAddressAndPort());
+        SummaryStatistics rf3StatsAfter = 
rf3Allocator.getAllocationRingOwnership(FBUtilities.getBroadcastAddressAndPort());
+
+        verifyImprovement(rf2StatsBefore, rf2StatsAfter);
+        verifyImprovement(rf3StatsBefore, rf3StatsAfter);
+    }
+
+    private void verifyImprovement(SummaryStatistics os, SummaryStatistics ns)
+    {
+        if (ns.getStandardDeviation() > os.getStandardDeviation())
+        {
+            fail(String.format("Token allocation unexpectedly increased 
standard deviation.\nStats before:\n%s\nStats after:\n%s", os, ns));
+        }
+    }
+
+    private void generateFakeEndpoints(TokenMetadata tmd, int numOldNodes, int 
numVNodes) throws UnknownHostException
+    {
+        tmd.clearUnsafe();
+        generateFakeEndpoints(tmd, numOldNodes, numVNodes, "0", "0");
+    }
+
+    private void generateFakeEndpoints(TokenMetadata tmd, int nodes, int 
vnodes, String dc, String rack) throws UnknownHostException
+    {
+        System.out.printf("Adding %d nodes to dc=%s, rack=%s.%n", nodes, dc, 
rack);
+        IPartitioner p = tmd.partitioner;
+
+        for (int i = 1; i <= nodes; i++)
+        {
+            // leave .1 for myEndpoint
+            InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc 
+ '.' + rack + '.' + (i + 1));
+            List<Token> tokens = Lists.newArrayListWithCapacity(vnodes);
+            for (int j = 0; j < vnodes; ++j)
+                tokens.add(p.getRandomToken(rand));
+            tmd.updateNormalTokens(tokens, addr);
         }
-        
-        SummaryStatistics ns3 = TokenAllocation.replicatedOwnershipStats(tm, 
Keyspace.open(ks3).getReplicationStrategy(), dcaddr);
-        SummaryStatistics ns2 = TokenAllocation.replicatedOwnershipStats(tm, 
Keyspace.open(ks2).getReplicationStrategy(), dcaddr);
-        verifyImprovement(os3, ns3);
-        verifyImprovement(os2, ns2);
     }
 }
diff --git a/test/unit/org/apache/cassandra/tools/GenerateTokensTest.java 
b/test/unit/org/apache/cassandra/tools/GenerateTokensTest.java
new file mode 100644
index 0000000..0e90c85
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/GenerateTokensTest.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.dht.tokenallocator.TokenAllocation;
+
+
+public class GenerateTokensTest
+{
+
+    @Test
+    public void testMain()
+    {
+        for (int rf = 1; rf <= 5; ++rf)
+        {
+            for (int numTokens = 1; numTokens <= 16 ; ++numTokens)
+            {
+                GenerateTokens.main(new String[]{"-n", "15", "-t", 
String.valueOf(numTokens), "--rf", String.valueOf(rf)});
+                GenerateTokens.main(new String[]{"-n", "15", "-t", 
String.valueOf(numTokens), "--rf", String.valueOf(rf), "--partitioner", 
"Murmur3Partitioner"});
+                GenerateTokens.main(new String[]{"-n", "15", "-t", 
String.valueOf(numTokens), "--rf", String.valueOf(rf), "--partitioner", 
"RandomPartitioner"});
+                GenerateTokens.main(new String[]{"-n", "15", "-t", 
String.valueOf(numTokens), "--rf", String.valueOf(rf), "--racks", "15"});
+                if (rf <= 2)
+                    GenerateTokens.main(new String[]{"-n", "10", "-t", 
String.valueOf(numTokens), "--rf", String.valueOf(rf), "--racks", "5,5"});
+                if (rf <= 3)
+                    GenerateTokens.main(new String[]{"-n", "15", "-t", 
String.valueOf(numTokens), "--rf", String.valueOf(rf), "--racks", "5,5,5"});
+                if (rf <= 4)
+                    GenerateTokens.main(new String[]{"-n", "16", "-t", 
String.valueOf(numTokens), "--rf", String.valueOf(rf), "--racks", "4,4,4,4"});
+                if (rf == 5)
+                    GenerateTokens.main(new String[]{"-n", "15", "-t", 
String.valueOf(numTokens), "--rf", String.valueOf(rf), "--racks", "3,3,3,3,3"});
+            }
+        }
+        
Assert.assertFalse(LoggerFactory.getLogger(TokenAllocation.class).isWarnEnabled());
+    }
+
+}
diff --git a/tools/bin/generatetokens b/tools/bin/generatetokens
new file mode 100755
index 0000000..e0a6b31
--- /dev/null
+++ b/tools/bin/generatetokens
@@ -0,0 +1,54 @@
+#!/bin/sh
+
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+if [ "x$CASSANDRA_INCLUDE" = "x" ]; then
+    # Locations (in order) to use when searching for an include file.
+    for include in "`dirname "$0"`/cassandra.in.sh" \
+                   "$HOME/.cassandra.in.sh" \
+                   /usr/share/cassandra/cassandra.in.sh \
+                   /usr/local/share/cassandra/cassandra.in.sh \
+                   /opt/cassandra/cassandra.in.sh; do
+        if [ -r "$include" ]; then
+            . "$include"
+            break
+        fi
+    done
+elif [ -r "$CASSANDRA_INCLUDE" ]; then
+    . "$CASSANDRA_INCLUDE"
+fi
+
+if [ -z "$CLASSPATH" ]; then
+    echo "You must set the CLASSPATH var" >&2
+    exit 1
+fi
+
+if [ "x${MAX_HEAP_SIZE}" = "x" ]; then
+    MAX_HEAP_SIZE="256M"
+fi
+
+if [ "x${MAX_DIRECT_MEMORY}" = "x" ]; then
+    MAX_DIRECT_MEMORY="2G"
+fi
+
+JVM_OPTS="${JVM_OPTS} -Xmx${MAX_HEAP_SIZE} 
-XX:MaxDirectMemorySize=${MAX_DIRECT_MEMORY}"
+
+"${JAVA}" ${JAVA_AGENT} -ea -cp "${CLASSPATH}" ${JVM_OPTS} \
+        -Dcassandra.storagedir="${cassandra_storagedir}" \
+        -Dlogback.configurationFile=logback-tools.xml \
+        org.apache.cassandra.tools.GenerateTokens "$@"
+


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to