You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2014/07/07 19:34:36 UTC

[13/23] Introduce CQL support for stress tool

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/DistributionOffsetApache.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/DistributionOffsetApache.java b/tools/stress/src/org/apache/cassandra/stress/generate/DistributionOffsetApache.java
new file mode 100644
index 0000000..b0e41eb
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/DistributionOffsetApache.java
@@ -0,0 +1,80 @@
+package org.apache.cassandra.stress.generate;
+/*
+ * 
+ * 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.
+ * 
+ */
+
+
+import org.apache.commons.math3.distribution.AbstractRealDistribution;
+
+public class DistributionOffsetApache extends Distribution
+{
+
+    final AbstractRealDistribution delegate;
+    final long min, delta;
+
+    public DistributionOffsetApache(AbstractRealDistribution delegate, long min, long max)
+    {
+        this.delegate = delegate;
+        this.min = min;
+        this.delta = max - min;
+    }
+
+    public void setSeed(long seed)
+    {
+        delegate.reseedRandomGenerator(seed);
+    }
+
+    @Override
+    public long next()
+    {
+        return offset(min, delta, delegate.sample());
+    }
+
+    public double nextDouble()
+    {
+        return offsetDouble(min, delta, delegate.sample());
+    }
+
+    @Override
+    public long inverseCumProb(double cumProb)
+    {
+        return offset(min, delta, delegate.inverseCumulativeProbability(cumProb));
+    }
+
+    private long offset(long min, long delta, double val)
+    {
+        long r = (long) val;
+        if (r < 0)
+            r = 0;
+        if (r > delta)
+            r = delta;
+        return min + r;
+    }
+
+    private double offsetDouble(long min, long delta, double r)
+    {
+        if (r < 0)
+            r = 0;
+        if (r > delta)
+            r = delta;
+        return min + r;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/Partition.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/Partition.java b/tools/stress/src/org/apache/cassandra/stress/generate/Partition.java
new file mode 100644
index 0000000..856d550
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/Partition.java
@@ -0,0 +1,343 @@
+package org.apache.cassandra.stress.generate;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.stress.generate.values.Generator;
+
+// a partition is re-used to reduce garbage generation, as is its internal RowIterator
+public class Partition
+{
+
+    private long idseed;
+    private final Object[] partitionKey;
+    private final PartitionGenerator generator;
+    private final RowIterator iterator;
+
+    public Partition(PartitionGenerator generator)
+    {
+        this.generator = generator;
+        this.partitionKey = new Object[generator.partitionKey.size()];
+        if (generator.clusteringComponents.size() > 0)
+            iterator = new MultiRowIterator();
+        else
+            iterator = new SingleRowIterator();
+    }
+
+    void setSeed(long seed)
+    {
+        long idseed = 0;
+        for (int i = 0 ; i < partitionKey.length ; i++)
+        {
+            Generator generator = this.generator.partitionKey.get(i);
+            // set the partition key seed based on the current work item we're processing
+            generator.setSeed(seed);
+            Object key = generator.generate();
+            partitionKey[i] = key;
+            // then contribute this value to the data seed
+            idseed = seed(key, generator.type, idseed);
+        }
+        this.idseed = idseed;
+    }
+
+    public RowIterator iterator(double useChance)
+    {
+        iterator.reset(useChance, 0);
+        return iterator;
+    }
+
+    public RowIterator iterator(int targetCount)
+    {
+        iterator.reset(Double.NaN, targetCount);
+        return iterator;
+    }
+
+    class SingleRowIterator extends RowIterator
+    {
+        boolean done;
+
+        void reset(double useChance, int targetCount)
+        {
+            done = false;
+        }
+
+        public Iterable<Row> batch(double ratio)
+        {
+            if (done)
+                return Collections.emptyList();
+            for (int i = 0 ; i < row.row.length ; i++)
+            {
+                Generator gen = generator.valueComponents.get(i);
+                gen.setSeed(idseed);
+                row.row[i] = gen.generate();
+            }
+            done = true;
+            return Collections.singleton(row);
+        }
+
+        public boolean done()
+        {
+            return done;
+        }
+    }
+
+    public abstract class RowIterator
+    {
+        // we reuse the row object to save garbage
+        final Row row = new Row(partitionKey, new Object[generator.clusteringComponents.size() + generator.valueComponents.size()]);
+
+        public abstract Iterable<Row> batch(double ratio);
+        abstract void reset(double useChance, int targetCount);
+
+        public abstract boolean done();
+
+        public Partition partition()
+        {
+            return Partition.this;
+        }
+    }
+
+    // permits iterating a random subset of the procedurally generated rows in this partition;  this is the only mechanism for visiting rows
+    // we maintain a stack of clustering components and their seeds; for each clustering component we visit, we generate all values it takes at that level,
+    // and then, using the average (total) number of children it takes we randomly choose whether or not we visit its children;
+    // if we do, we generate all possible values the children can take, and repeat the process. So at any one time we are using space proportional
+    // to C.N, where N is the average number of values each clustering component takes, as opposed to N^C total values in the partition.
+    class MultiRowIterator extends RowIterator
+    {
+
+        // probability any single row will be generated in this iteration
+        double useChance;
+        double expectedRowCount;
+
+        // the current seed in use at any given level; used to save recalculating it for each row, so we only need to recalc
+        // from prior row
+        final long[] clusteringSeeds = new long[generator.clusteringComponents.size()];
+        // the components remaining to be visited for each level of the current stack
+        final Queue<Object>[] clusteringComponents = new ArrayDeque[generator.clusteringComponents.size()];
+
+        // we want our chance of selection to be applied uniformly, so we compound the roll we make at each level
+        // so that we know with what chance we reached there, and we adjust our roll at that level by that amount
+        double[] chancemodifier = new double[generator.clusteringComponents.size()];
+        double[] rollmodifier = new double[generator.clusteringComponents.size()];
+
+        // reusable set for generating unique clustering components
+        final Set<Object> unique = new HashSet<>();
+        final Random random = new Random();
+
+        MultiRowIterator()
+        {
+            for (int i = 0 ; i < clusteringComponents.length ; i++)
+                clusteringComponents[i] = new ArrayDeque<>();
+            rollmodifier[0] = 1f;
+            chancemodifier[0] = generator.clusteringChildAverages[0];
+        }
+
+        void reset(double useChance, int targetCount)
+        {
+            generator.clusteringComponents.get(0).setSeed(idseed);
+            int firstComponentCount = (int) generator.clusteringComponents.get(0).clusteringDistribution.next();
+            this.expectedRowCount = firstComponentCount * generator.clusteringChildAverages[0];
+            if (Double.isNaN(useChance))
+                useChance = Math.max(0d, Math.min(1d, targetCount / expectedRowCount));
+
+            for (Queue<?> q : clusteringComponents)
+                q.clear();
+
+            this.useChance = useChance;
+            clusteringSeeds[0] = idseed;
+            clusteringComponents[0].add(this);
+            fill(clusteringComponents[0], firstComponentCount, generator.clusteringComponents.get(0));
+            advance(0, 1f);
+        }
+
+        void fill(int component)
+        {
+            long seed = clusteringSeeds[component - 1];
+            Generator gen = generator.clusteringComponents.get(component);
+            gen.setSeed(seed);
+            clusteringSeeds[component] = seed(clusteringComponents[component - 1].peek(), generator.clusteringComponents.get(component - 1).type, seed);
+            fill(clusteringComponents[component], (int) gen.clusteringDistribution.next(), gen);
+        }
+
+        void fill(Queue<Object> queue, int count, Generator generator)
+        {
+            if (count == 1)
+            {
+                queue.add(generator.generate());
+            }
+            else
+            {
+                unique.clear();
+                for (int i = 0 ; i < count ; i++)
+                {
+                    Object next = generator.generate();
+                    if (unique.add(next))
+                        queue.add(next);
+                }
+            }
+        }
+
+        private boolean advance(double continueChance)
+        {
+            // we always start at the leaf level
+            int depth = clusteringComponents.length - 1;
+            // fill the row with the position we *were* at (unless pre-start)
+            for (int i = clusteringSeeds.length ; i < row.row.length ; i++)
+            {
+                Generator gen = generator.valueComponents.get(i - clusteringSeeds.length);
+                long seed = clusteringSeeds[depth];
+                seed = seed(clusteringComponents[depth].peek(), generator.clusteringComponents.get(depth).type, seed);
+                gen.setSeed(seed);
+                row.row[i] = gen.generate();
+            }
+            clusteringComponents[depth].poll();
+
+            return advance(depth, continueChance);
+        }
+
+        private boolean advance(int depth, double continueChance)
+        {
+            // advance the leaf component
+            clusteringComponents[depth].poll();
+            while (true)
+            {
+                if (clusteringComponents[depth].isEmpty())
+                {
+                    if (depth == 0)
+                        return false;
+                    depth--;
+                    clusteringComponents[depth].poll();
+                    continue;
+                }
+
+                // the chance of descending is the uniform use chance, multiplied by the number of children
+                // we would on average generate (so if we have a 0.1 use chance, but should generate 10 children
+                // then we will always descend), multiplied by 1/(compound roll), where (compound roll) is the
+                // chance with which we reached this depth, i.e. if we already beat 50/50 odds, we double our
+                // chance of beating this next roll
+                double thischance = useChance * chancemodifier[depth];
+                if (thischance > 0.999f || thischance >= random.nextDouble())
+                {
+                    row.row[depth] = clusteringComponents[depth].peek();
+                    depth++;
+                    if (depth == clusteringComponents.length)
+                        break;
+                    rollmodifier[depth] = rollmodifier[depth - 1] / Math.min(1d, thischance);
+                    chancemodifier[depth] = generator.clusteringChildAverages[depth] * rollmodifier[depth];
+                    fill(depth);
+                    continue;
+                }
+
+                clusteringComponents[depth].poll();
+            }
+
+            return continueChance >= 1.0d || continueChance >= random.nextDouble();
+        }
+
+        public Iterable<Row> batch(final double ratio)
+        {
+            final double continueChance = 1d - (Math.pow(ratio, expectedRowCount * useChance));
+            return new Iterable<Row>()
+            {
+                public Iterator<Row> iterator()
+                {
+                    return new Iterator<Row>()
+                    {
+                        boolean hasNext = true;
+                        public boolean hasNext()
+                        {
+                            return hasNext;
+                        }
+
+                        public Row next()
+                        {
+                            hasNext = advance(continueChance);
+                            return row;
+                        }
+
+                        public void remove()
+                        {
+                            throw new UnsupportedOperationException();
+                        }
+                    };
+                }
+            };
+        }
+
+        public boolean done()
+        {
+            return clusteringComponents[0].isEmpty();
+        }
+
+        public Partition partition()
+        {
+            return Partition.this;
+        }
+    }
+
+    public String getKeyAsString()
+    {
+        StringBuilder sb = new StringBuilder();
+        int i = 0;
+        for (Object key : partitionKey)
+        {
+            if (i > 0)
+                sb.append("|");
+            AbstractType type = generator.partitionKey.get(i++).type;
+            sb.append(type.getString(type.decompose(key)));
+        }
+        return sb.toString();
+    }
+
+    static long seed(Object object, AbstractType type, long seed)
+    {
+        if (object instanceof ByteBuffer)
+        {
+            ByteBuffer buf = (ByteBuffer) object;
+            for (int i = buf.position() ; i < buf.limit() ; i++)
+                seed = (31 * seed) + buf.get(i);
+            return seed;
+        }
+        else if (object instanceof String)
+        {
+            String str = (String) object;
+            for (int i = 0 ; i < str.length() ; i++)
+                seed = (31 * seed) + str.charAt(i);
+            return seed;
+        }
+        else if (object instanceof Number)
+        {
+            return (seed * 31) + ((Number) object).longValue();
+        }
+        else if (object instanceof UUID)
+        {
+            return seed * 31 + (((UUID) object).getLeastSignificantBits() ^ ((UUID) object).getMostSignificantBits());
+        }
+        else
+        {
+            return seed(type.decompose(object), BytesType.instance, seed);
+        }
+    }
+
+    public Object getPartitionKey(int i)
+    {
+        return partitionKey[i];
+    }
+
+    // used for thrift smart routing - if it's a multi-part key we don't try to route correctly right now
+    public ByteBuffer getToken()
+    {
+        return generator.partitionKey.get(0).type.decompose(partitionKey[0]);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/PartitionGenerator.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/PartitionGenerator.java b/tools/stress/src/org/apache/cassandra/stress/generate/PartitionGenerator.java
new file mode 100644
index 0000000..78002fb
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/PartitionGenerator.java
@@ -0,0 +1,80 @@
+package org.apache.cassandra.stress.generate;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.stress.generate.values.Generator;
+
+public class PartitionGenerator
+{
+
+    public final double maxRowCount;
+    final List<Generator> partitionKey;
+    final List<Generator> clusteringComponents;
+    final List<Generator> valueComponents;
+    final int[] clusteringChildAverages;
+
+    private final Map<String, Integer> indexMap;
+
+    final List<Partition> recyclable = new ArrayList<>();
+    int partitionsInUse = 0;
+
+    public void reset()
+    {
+        partitionsInUse = 0;
+    }
+
+    public PartitionGenerator(List<Generator> partitionKey, List<Generator> clusteringComponents, List<Generator> valueComponents)
+    {
+        this.partitionKey = partitionKey;
+        this.clusteringComponents = clusteringComponents;
+        this.valueComponents = valueComponents;
+        this.clusteringChildAverages = new int[clusteringComponents.size()];
+        for (int i = clusteringChildAverages.length - 1 ; i >= 0 ; i--)
+            clusteringChildAverages[i] = (int) (i < (clusteringChildAverages.length - 1) ? clusteringComponents.get(i + 1).clusteringDistribution.average() * clusteringChildAverages[i + 1] : 1);
+        double maxRowCount = 1d;
+        for (Generator component : clusteringComponents)
+            maxRowCount *= component.clusteringDistribution.maxValue();
+        this.maxRowCount = maxRowCount;
+        this.indexMap = new HashMap<>();
+        int i = 0;
+        for (Generator generator : partitionKey)
+            indexMap.put(generator.name, --i);
+        i = 0;
+        for (Generator generator : Iterables.concat(clusteringComponents, valueComponents))
+            indexMap.put(generator.name, i++);
+    }
+
+    public int indexOf(String name)
+    {
+        Integer i = indexMap.get(name);
+        if (i == null)
+            throw new NoSuchElementException();
+        return i;
+    }
+
+    public Partition generate(long seed)
+    {
+        if (recyclable.size() <= partitionsInUse || recyclable.get(partitionsInUse) == null)
+            recyclable.add(new Partition(this));
+
+        Partition partition = recyclable.get(partitionsInUse++);
+        partition.setSeed(seed);
+        return partition;
+    }
+
+    public ByteBuffer convert(int c, Object v)
+    {
+        if (c < 0)
+            return partitionKey.get(-1-c).type.decompose(v);
+        if (c < clusteringComponents.size())
+            return clusteringComponents.get(c).type.decompose(v);
+        return valueComponents.get(c - clusteringComponents.size()).type.decompose(v);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/RatioDistribution.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/RatioDistribution.java b/tools/stress/src/org/apache/cassandra/stress/generate/RatioDistribution.java
new file mode 100644
index 0000000..fb5a373
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/RatioDistribution.java
@@ -0,0 +1,25 @@
+package org.apache.cassandra.stress.generate;
+
+public class RatioDistribution
+{
+
+    final Distribution distribution;
+    final double divisor;
+
+    public RatioDistribution(Distribution distribution, double divisor)
+    {
+        this.distribution = distribution;
+        this.divisor = divisor;
+    }
+
+    // yields a value between 0 and 1
+    public double next()
+    {
+        return Math.max(0f, Math.min(1f, distribution.nextDouble() / divisor));
+    }
+
+    public double max()
+    {
+        return Math.min(1d, distribution.maxValue() / divisor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/RatioDistributionFactory.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/RatioDistributionFactory.java b/tools/stress/src/org/apache/cassandra/stress/generate/RatioDistributionFactory.java
new file mode 100644
index 0000000..16474d8
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/RatioDistributionFactory.java
@@ -0,0 +1,31 @@
+package org.apache.cassandra.stress.generate;
+/*
+ * 
+ * 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.
+ * 
+ */
+
+
+import java.io.Serializable;
+
+public interface RatioDistributionFactory extends Serializable
+{
+
+    RatioDistribution get();
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/Row.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/Row.java b/tools/stress/src/org/apache/cassandra/stress/generate/Row.java
new file mode 100644
index 0000000..d3b9a2a
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/Row.java
@@ -0,0 +1,22 @@
+package org.apache.cassandra.stress.generate;
+
+public class Row
+{
+
+    final Object[] partitionKey;
+    final Object[] row;
+
+    public Row(Object[] partitionKey, Object[] row)
+    {
+        this.partitionKey = partitionKey;
+        this.row = row;
+    }
+
+    public Object get(int column)
+    {
+        if (column < 0)
+            return partitionKey[-1-column];
+        return row[column];
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/SeedGenerator.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/SeedGenerator.java b/tools/stress/src/org/apache/cassandra/stress/generate/SeedGenerator.java
new file mode 100644
index 0000000..6c1bc7f
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/SeedGenerator.java
@@ -0,0 +1,8 @@
+package org.apache.cassandra.stress.generate;
+
+public interface SeedGenerator
+{
+
+    long next(long workIndex);
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/SeedRandomGenerator.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/SeedRandomGenerator.java b/tools/stress/src/org/apache/cassandra/stress/generate/SeedRandomGenerator.java
new file mode 100644
index 0000000..bbaaeb8
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/SeedRandomGenerator.java
@@ -0,0 +1,33 @@
+package org.apache.cassandra.stress.generate;
+
+public class SeedRandomGenerator implements SeedGenerator
+{
+
+    final Distribution distribution;
+    final Distribution clustering;
+
+    private long next;
+    private int count;
+
+    public SeedRandomGenerator(Distribution distribution, Distribution clustering)
+    {
+        this.distribution = distribution;
+        this.clustering = clustering;
+    }
+
+    public long next(long workIndex)
+    {
+        if (count == 0)
+        {
+            next = distribution.next();
+            count = (int) clustering.next();
+        }
+        long result = next;
+        count--;
+        if (next == distribution.maxValue())
+            next = distribution.minValue();
+        else
+            next++;
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/SeedSeriesGenerator.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/SeedSeriesGenerator.java b/tools/stress/src/org/apache/cassandra/stress/generate/SeedSeriesGenerator.java
new file mode 100644
index 0000000..27967d2
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/SeedSeriesGenerator.java
@@ -0,0 +1,21 @@
+package org.apache.cassandra.stress.generate;
+
+public class SeedSeriesGenerator implements SeedGenerator
+{
+
+    final long min;
+    final long count;
+
+    public SeedSeriesGenerator(long min, long max)
+    {
+        if (min > max)
+            throw new IllegalStateException();
+        this.min = min;
+        this.count = 1 + max - min;
+    }
+
+    public long next(long workIndex)
+    {
+        return min + (workIndex % count);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Booleans.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Booleans.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Booleans.java
new file mode 100644
index 0000000..b1d84d6
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Booleans.java
@@ -0,0 +1,37 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.BooleanType;
+
+public class Booleans extends Generator<Boolean>
+{
+    public Booleans(String name, GeneratorConfig config)
+    {
+        super(BooleanType.instance, config, name);
+    }
+
+    @Override
+    public Boolean generate()
+    {
+        return identityDistribution.next() % 1 == 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Bytes.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Bytes.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Bytes.java
new file mode 100644
index 0000000..2a5bddf
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Bytes.java
@@ -0,0 +1,54 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.BytesType;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+public class Bytes extends Generator<ByteBuffer>
+{
+    private final byte[] bytes;
+    private final Random rand = new Random();
+
+    public Bytes(String name, GeneratorConfig config)
+    {
+        super(BytesType.instance, config, name);
+        bytes = new byte[(int) sizeDistribution.maxValue()];
+    }
+
+    @Override
+    public ByteBuffer generate()
+    {
+        long seed = identityDistribution.next();
+        sizeDistribution.setSeed(seed);
+        rand.setSeed(~seed);
+        int size = (int) sizeDistribution.next();
+        for (int i = 0; i < size; )
+            for (int v = rand.nextInt(),
+                 n = Math.min(size - i, Integer.SIZE/Byte.SIZE);
+                 n-- > 0; v >>= Byte.SIZE)
+                bytes[i++] = (byte)v;
+        return ByteBuffer.wrap(Arrays.copyOf(bytes, size));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Dates.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Dates.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Dates.java
new file mode 100644
index 0000000..7d36be2
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Dates.java
@@ -0,0 +1,46 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import java.util.Date;
+
+import org.apache.cassandra.db.marshal.DateType;
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.stress.settings.OptionDistribution;
+
+public class Dates extends Generator<Date>
+{
+    public Dates(String name, GeneratorConfig config)
+    {
+        super(DateType.instance, config, name);
+    }
+
+    @Override
+    public Date generate()
+    {
+        return new Date(identityDistribution.next());
+    }
+
+    DistributionFactory defaultIdentityDistribution()
+    {
+        return OptionDistribution.get("uniform(1.." + Long.toString(50L*365L*24L*60L*60L*1000L) + ")");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Doubles.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Doubles.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Doubles.java
new file mode 100644
index 0000000..76e983d
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Doubles.java
@@ -0,0 +1,37 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+
+public class Doubles extends Generator<Double>
+{
+    public Doubles(String name, GeneratorConfig config)
+    {
+        super(DoubleType.instance, config, name);
+    }
+
+    @Override
+    public Double generate()
+    {
+        return identityDistribution.nextDouble();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Floats.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Floats.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Floats.java
new file mode 100644
index 0000000..8e23c11
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Floats.java
@@ -0,0 +1,37 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.FloatType;
+
+public class Floats extends Generator<Float>
+{
+    public Floats(String name, GeneratorConfig config)
+    {
+        super(FloatType.instance, config, name);
+    }
+
+    @Override
+    public Float generate()
+    {
+        return (float) identityDistribution.nextDouble();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Generator.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Generator.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Generator.java
new file mode 100644
index 0000000..13343de
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Generator.java
@@ -0,0 +1,50 @@
+package org.apache.cassandra.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.stress.generate.Distribution;
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.stress.settings.OptionDistribution;
+
+public abstract class Generator<T>
+{
+
+    public final String name;
+    public final AbstractType<T> type;
+    final long salt;
+    final Distribution identityDistribution;
+    final Distribution sizeDistribution;
+    public final Distribution clusteringDistribution;
+
+    public Generator(AbstractType<T> type, GeneratorConfig config, String name)
+    {
+        this.type = type;
+        this.name = name;
+        this.salt = config.salt;
+        this.identityDistribution = config.getIdentityDistribution(defaultIdentityDistribution());
+        this.sizeDistribution = config.getSizeDistribution(defaultSizeDistribution());
+        this.clusteringDistribution = config.getClusteringDistribution(defaultClusteringDistribution());
+    }
+
+    public void setSeed(long seed)
+    {
+        identityDistribution.setSeed(seed ^ salt);
+        clusteringDistribution.setSeed(seed ^ ~salt);
+    }
+
+    public abstract T generate();
+
+    DistributionFactory defaultIdentityDistribution()
+    {
+        return OptionDistribution.get("uniform(1..100B)");
+    }
+
+    DistributionFactory defaultSizeDistribution()
+    {
+        return OptionDistribution.get("uniform(4..8)");
+    }
+
+    DistributionFactory defaultClusteringDistribution()
+    {
+        return OptionDistribution.get("fixed(1)");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/GeneratorConfig.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/GeneratorConfig.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/GeneratorConfig.java
new file mode 100644
index 0000000..8f7b2ea
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/GeneratorConfig.java
@@ -0,0 +1,68 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.stress.generate.Distribution;
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.MurmurHash;
+
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Map;
+
+public class GeneratorConfig implements Serializable
+{
+    public final long salt;
+
+    private final DistributionFactory clusteringDistributions;
+    private final DistributionFactory sizeDistributions;
+    private final DistributionFactory identityDistributions;
+
+    public GeneratorConfig(String seedStr, DistributionFactory clusteringDistributions, DistributionFactory sizeDistributions, DistributionFactory identityDistributions)
+    {
+        this.clusteringDistributions = clusteringDistributions;
+        this.sizeDistributions = sizeDistributions;
+        this.identityDistributions = identityDistributions;
+        ByteBuffer buf = ByteBufferUtil.bytes(seedStr);
+        long[] hash = new long[2];
+        MurmurHash.hash3_x64_128(buf, buf.position(), buf.remaining(), 0, hash);
+        salt = hash[0];
+    }
+
+    Distribution getClusteringDistribution(DistributionFactory deflt)
+    {
+        return (clusteringDistributions == null ? deflt : clusteringDistributions).get();
+    }
+
+    Distribution getIdentityDistribution(DistributionFactory deflt)
+    {
+        return (identityDistributions == null ? deflt : identityDistributions).get();
+    }
+
+    Distribution getSizeDistribution(DistributionFactory deflt)
+    {
+        return (sizeDistributions == null ? deflt : sizeDistributions).get();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/HexBytes.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/HexBytes.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/HexBytes.java
new file mode 100644
index 0000000..db46bac
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/HexBytes.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.stress.generate.values;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.cassandra.db.marshal.BytesType;
+
+public class HexBytes extends Generator<ByteBuffer>
+{
+    private final byte[] bytes;
+
+    public HexBytes(String name, GeneratorConfig config)
+    {
+        super(BytesType.instance, config, name);
+        bytes = new byte[(int) sizeDistribution.maxValue()];
+    }
+
+    @Override
+    public ByteBuffer generate()
+    {
+        long seed = identityDistribution.next();
+        sizeDistribution.setSeed(seed);
+        int size = (int) sizeDistribution.next();
+        for (int i = 0 ; i < size ; i +=16)
+        {
+            long value = identityDistribution.next();
+            for (int j = 0 ; j < 16 && i + j < size ; j++)
+            {
+                int v = (int) (value & 15);
+                bytes[i + j] = (byte) ((v < 10 ? '0' : 'A') + v);
+                value >>>= 4;
+            }
+        }
+        return ByteBuffer.wrap(Arrays.copyOf(bytes, size));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/HexStrings.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/HexStrings.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/HexStrings.java
new file mode 100644
index 0000000..ce65b8a
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/HexStrings.java
@@ -0,0 +1,55 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import java.util.Random;
+
+import org.apache.cassandra.db.marshal.UTF8Type;
+
+public class HexStrings extends Generator<String>
+{
+    private final char[] chars;
+
+    public HexStrings(String name, GeneratorConfig config)
+    {
+        super(UTF8Type.instance, config, name);
+        chars = new char[(int) sizeDistribution.maxValue()];
+    }
+
+    @Override
+    public String generate()
+    {
+        long seed = identityDistribution.next();
+        sizeDistribution.setSeed(seed);
+        int size = (int) sizeDistribution.next();
+        for (int i = 0 ; i < size ; i +=16)
+        {
+            long value = identityDistribution.next();
+            for (int j = 0 ; j < 16 && i + j < size ; j++)
+            {
+                int v = (int) (value & 15);
+                chars[i + j] = (char) ((v < 10 ? '0' : 'A') + v);
+                value >>>= 4;
+            }
+        }
+        return new String(chars, 0, size);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Inets.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Inets.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Inets.java
new file mode 100644
index 0000000..334d73c
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Inets.java
@@ -0,0 +1,57 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.cassandra.db.marshal.InetAddressType;
+
+
+public class Inets extends Generator<InetAddress>
+{
+    final byte[] buf;
+    public Inets(String name, GeneratorConfig config)
+    {
+        super(InetAddressType.instance, config, name);
+        buf = new byte[4];
+    }
+
+    @Override
+    public InetAddress generate()
+    {
+        int val = (int) identityDistribution.next();
+
+        buf[0] = (byte)(val >>> 24);
+        buf[1] = (byte)(val >>> 16);
+        buf[2] = (byte)(val >>> 8);
+        buf[3] = (byte)val;
+
+        try
+        {
+            return InetAddress.getByAddress(buf);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Integers.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Integers.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Integers.java
new file mode 100644
index 0000000..8b9b33a
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Integers.java
@@ -0,0 +1,38 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.Int32Type;
+
+public class Integers extends Generator<Integer>
+{
+
+    public Integers(String name, GeneratorConfig config)
+    {
+        super(Int32Type.instance, config, name);
+    }
+
+    @Override
+    public Integer generate()
+    {
+        return (int) identityDistribution.next();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Lists.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Lists.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Lists.java
new file mode 100644
index 0000000..d188f7e
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Lists.java
@@ -0,0 +1,55 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.cassandra.db.marshal.ListType;
+
+public class Lists extends Generator<List>
+{
+    final Generator valueType;
+    final Object[] buffer;
+
+    public Lists(String name, Generator valueType, GeneratorConfig config)
+    {
+        super(ListType.getInstance(valueType.type), config, name);
+        this.valueType = valueType;
+        buffer = new Object[(int) sizeDistribution.maxValue()];
+    }
+
+    public void setSeed(long seed)
+    {
+        super.setSeed(seed);
+        valueType.setSeed(seed * 31);
+    }
+
+    @Override
+    public List generate()
+    {
+        int size = (int) sizeDistribution.next();
+        for (int i = 0 ; i < size ; i++)
+            buffer[i] = valueType.generate();
+        return com.google.common.collect.Lists.newArrayList(Arrays.copyOf(buffer, size));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Longs.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Longs.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Longs.java
new file mode 100644
index 0000000..0584ed1
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Longs.java
@@ -0,0 +1,37 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.LongType;
+
+public class Longs extends Generator<Long>
+{
+    public Longs(String name, GeneratorConfig config)
+    {
+        super(LongType.instance, config, name);
+    }
+
+    @Override
+    public Long generate()
+    {
+        return identityDistribution.next();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Sets.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Sets.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Sets.java
new file mode 100644
index 0000000..48bf293
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Sets.java
@@ -0,0 +1,54 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.cassandra.db.marshal.SetType;
+
+public class Sets extends Generator<Set>
+{
+    final Generator valueType;
+
+    public Sets(String name, Generator valueType, GeneratorConfig config)
+    {
+        super(SetType.getInstance(valueType.type), config, name);
+        this.valueType = valueType;
+    }
+
+    public void setSeed(long seed)
+    {
+        super.setSeed(seed);
+        valueType.setSeed(seed * 31);
+    }
+
+    @Override
+    public Set generate()
+    {
+        final Set set = new HashSet();
+        int size = (int) sizeDistribution.next();
+        for (int i = 0 ; i < size ; i++)
+            set.add(valueType.generate());
+        return set;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/Strings.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Strings.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Strings.java
new file mode 100644
index 0000000..e01ff20
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Strings.java
@@ -0,0 +1,49 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import java.util.Random;
+
+import org.apache.cassandra.db.marshal.UTF8Type;
+
+public class Strings extends Generator<String>
+{
+    private final char[] chars;
+    private final Random rnd = new Random();
+
+    public Strings(String name, GeneratorConfig config)
+    {
+        super(UTF8Type.instance, config, name);
+        chars = new char[(int) sizeDistribution.maxValue()];
+    }
+
+    @Override
+    public String generate()
+    {
+        long seed = identityDistribution.next();
+        sizeDistribution.setSeed(seed);
+        rnd.setSeed(~seed);
+        int size = (int) sizeDistribution.next();
+        for (int i = 0 ; i < size ; i++)
+            chars[i] = (char) (32 +rnd.nextInt(128-32));
+        return new String(chars, 0, size);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
new file mode 100644
index 0000000..714959d
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
@@ -0,0 +1,51 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.utils.UUIDGen;
+
+import java.util.UUID;
+
+public class TimeUUIDs extends Generator<UUID>
+{
+    final Dates dateGen;
+    final long clockSeqAndNode;
+
+    public TimeUUIDs(String name, GeneratorConfig config)
+    {
+        super(TimeUUIDType.instance, config, name);
+        dateGen = new Dates(name, config);
+        clockSeqAndNode = config.salt;
+    }
+
+    public void setSeed(long seed)
+    {
+        dateGen.setSeed(seed);
+    }
+
+    @Override
+    public UUID generate()
+    {
+        return UUIDGen.getTimeUUID(dateGen.generate().getTime(), clockSeqAndNode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generate/values/UUIDs.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/UUIDs.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/UUIDs.java
new file mode 100644
index 0000000..e8d6501
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/UUIDs.java
@@ -0,0 +1,39 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import java.util.UUID;
+
+import org.apache.cassandra.db.marshal.UUIDType;
+
+public class UUIDs extends Generator<UUID>
+{
+    public UUIDs(String name, GeneratorConfig config)
+    {
+        super(UUIDType.instance, config, name);
+    }
+
+    @Override
+    public UUID generate()
+    {
+        return new UUID(identityDistribution.next(), identityDistribution.next());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGen.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGen.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGen.java
deleted file mode 100644
index 334506d..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGen.java
+++ /dev/null
@@ -1,39 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-public abstract class DataGen
-{
-
-    public abstract void generate(ByteBuffer fill, long index, ByteBuffer seed);
-    public abstract boolean isDeterministic();
-
-    public void generate(List<ByteBuffer> fills, long index, ByteBuffer seed)
-    {
-        for (ByteBuffer fill : fills)
-            generate(fill, index++, seed);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenBytesRandom.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenBytesRandom.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenBytesRandom.java
deleted file mode 100644
index 1dc77b4..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenBytesRandom.java
+++ /dev/null
@@ -1,45 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import java.nio.ByteBuffer;
-import java.util.Random;
-
-public class DataGenBytesRandom extends DataGen
-{
-
-    private final Random rnd = new Random();
-
-    @Override
-    public void generate(ByteBuffer fill, long index, ByteBuffer seed)
-    {
-        fill.clear();
-        rnd.nextBytes(fill.array());
-    }
-
-    @Override
-    public boolean isDeterministic()
-    {
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenFactory.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenFactory.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenFactory.java
deleted file mode 100644
index dadb792..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenFactory.java
+++ /dev/null
@@ -1,30 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import java.io.Serializable;
-
-public interface DataGenFactory extends Serializable
-{
-    DataGen get();
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHex.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHex.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHex.java
deleted file mode 100644
index 124a07b..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHex.java
+++ /dev/null
@@ -1,60 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import java.nio.ByteBuffer;
-
-public abstract class DataGenHex extends DataGen
-{
-
-    abstract long next(long operationIndex);
-
-    @Override
-    public final void generate(ByteBuffer fill, long operationIndex, ByteBuffer seed)
-    {
-        fill.clear();
-        fillKeyStringBytes(next(operationIndex), fill.array());
-    }
-
-    public static void fillKeyStringBytes(long key, byte[] fill)
-    {
-        int ub = fill.length - 1;
-        int offset = 0;
-        while (key != 0)
-        {
-            int digit = ((int) key) & 15;
-            key >>>= 4;
-            fill[ub - offset++] = digit(digit);
-        }
-        while (offset < fill.length)
-            fill[ub - offset++] = '0';
-    }
-
-    // needs to be UTF-8, but for these chars there is no difference
-    private static byte digit(int num)
-    {
-        if (num < 10)
-            return (byte)('0' + num);
-        return (byte)('A' + (num - 10));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromDistribution.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromDistribution.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromDistribution.java
deleted file mode 100644
index 25284dc..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromDistribution.java
+++ /dev/null
@@ -1,66 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import org.apache.commons.math3.distribution.NormalDistribution;
-import org.apache.commons.math3.distribution.UniformRealDistribution;
-
-public class DataGenHexFromDistribution extends DataGenHex
-{
-
-    final Distribution distribution;
-
-    public DataGenHexFromDistribution(Distribution distribution)
-    {
-        this.distribution = distribution;
-    }
-
-    @Override
-    public boolean isDeterministic()
-    {
-        return false;
-    }
-
-    @Override
-    long next(long operationIndex)
-    {
-        return distribution.next();
-    }
-
-    public static DataGenHex buildGaussian(long minKey, long maxKey, double stdevsToLimit)
-    {
-        double midRange = (maxKey + minKey) / 2d;
-        double halfRange = (maxKey - minKey) / 2d;
-        return new DataGenHexFromDistribution(new DistributionBoundApache(new NormalDistribution(midRange, halfRange / stdevsToLimit), minKey, maxKey));
-    }
-
-    public static DataGenHex buildGaussian(long minKey, long maxKey, double mean, double stdev)
-    {
-        return new DataGenHexFromDistribution(new DistributionBoundApache(new NormalDistribution(mean, stdev), minKey, maxKey));
-    }
-
-    public static DataGenHex buildUniform(long minKey, long maxKey)
-    {
-        return new DataGenHexFromDistribution(new DistributionBoundApache(new UniformRealDistribution(minKey, maxKey), minKey, maxKey));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromOpIndex.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromOpIndex.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromOpIndex.java
deleted file mode 100644
index e6bda68..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromOpIndex.java
+++ /dev/null
@@ -1,48 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-public class DataGenHexFromOpIndex extends DataGenHex
-{
-
-    final long minKey;
-    final long maxKey;
-
-    public DataGenHexFromOpIndex(long minKey, long maxKey)
-    {
-        this.minKey = minKey;
-        this.maxKey = maxKey;
-    }
-
-    @Override
-    public boolean isDeterministic()
-    {
-        return true;
-    }
-
-    @Override
-    long next(long operationIndex)
-    {
-        long range = maxKey + 1 - minKey;
-        return Math.abs((operationIndex % range) + minKey);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringDictionary.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringDictionary.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringDictionary.java
deleted file mode 100644
index 73814cb..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringDictionary.java
+++ /dev/null
@@ -1,107 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.math3.distribution.EnumeratedDistribution;
-import org.apache.commons.math3.util.Pair;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-public class DataGenStringDictionary extends DataGen
-{
-
-    private final byte space = ' ';
-    private final EnumeratedDistribution<byte[]> words;
-
-    public DataGenStringDictionary(EnumeratedDistribution<byte[]> wordDistribution)
-    {
-        words = wordDistribution;
-    }
-
-    @Override
-    public void generate(ByteBuffer fill, long index, ByteBuffer seed)
-    {
-        fill(fill);
-    }
-
-    @Override
-    public void generate(List<ByteBuffer> fills, long index, ByteBuffer seed)
-    {
-        for (int i = 0 ; i < fills.size() ; i++)
-            fill(fills.get(0));
-    }
-
-    private void fill(ByteBuffer fill)
-    {
-        fill.clear();
-        byte[] trg = fill.array();
-        int i = 0;
-        while (i < trg.length)
-        {
-            if (i > 0)
-                trg[i++] = space;
-            byte[] src = words.sample();
-            System.arraycopy(src, 0, trg, i, Math.min(src.length, trg.length - i));
-            i += src.length;
-        }
-    }
-
-    @Override
-    public boolean isDeterministic()
-    {
-        return false;
-    }
-
-    public static DataGenFactory getFactory(File file) throws IOException
-    {
-        final List<Pair<byte[], Double>> words = new ArrayList<>();
-        try (final BufferedReader reader = new BufferedReader(new FileReader(file)))
-        {
-            String line;
-            while ( null != (line = reader.readLine()) )
-            {
-                String[] pair = line.split(" +");
-                if (pair.length != 2)
-                    throw new IllegalArgumentException("Invalid record in dictionary: \"" + line + "\"");
-                words.add(new Pair<>(pair[1].getBytes(UTF_8), Double.parseDouble(pair[0])));
-            }
-            final EnumeratedDistribution<byte[]> dist = new EnumeratedDistribution<byte[]>(words);
-            return new DataGenFactory()
-            {
-                @Override
-                public DataGen get()
-                {
-                    return new DataGenStringDictionary(dist);
-                }
-            };
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringRepeats.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringRepeats.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringRepeats.java
deleted file mode 100644
index ad7cb24..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringRepeats.java
+++ /dev/null
@@ -1,90 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.cassandra.utils.FBUtilities;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-public class DataGenStringRepeats extends DataGen
-{
-
-    private static final ConcurrentHashMap<Integer, ConcurrentHashMap<Long, byte[]>> CACHE_LOOKUP = new ConcurrentHashMap<>();
-
-    private final ConcurrentHashMap<Long, byte[]> cache;
-    private final int repeatFrequency;
-    public DataGenStringRepeats(int repeatFrequency)
-    {
-        if (!CACHE_LOOKUP.containsKey(repeatFrequency))
-            CACHE_LOOKUP.putIfAbsent(repeatFrequency, new ConcurrentHashMap<Long, byte[]>());
-        cache = CACHE_LOOKUP.get(repeatFrequency);
-        this.repeatFrequency = repeatFrequency;
-    }
-
-    @Override
-    public void generate(ByteBuffer fill, long index, ByteBuffer seed)
-    {
-        fill(fill, index, 0, seed);
-    }
-
-    @Override
-    public void generate(List<ByteBuffer> fills, long index, ByteBuffer seed)
-    {
-        for (int i = 0 ; i < fills.size() ; i++)
-        {
-            fill(fills.get(i), index, i, seed);
-        }
-    }
-
-    private void fill(ByteBuffer fill, long index, int column, ByteBuffer seed)
-    {
-        fill.clear();
-        byte[] trg = fill.array();
-        byte[] src = getData(index, column, seed);
-        for (int j = 0 ; j < trg.length ; j += src.length)
-            System.arraycopy(src, 0, trg, j, Math.min(src.length, trg.length - j));
-    }
-
-    private byte[] getData(long index, int column, ByteBuffer seed)
-    {
-        final long key = ((long)column * repeatFrequency) + ((seed == null ? index : Math.abs(seed.hashCode())) % repeatFrequency);
-        byte[] r = cache.get(key);
-        if (r != null)
-            return r;
-        MessageDigest md = FBUtilities.threadLocalMD5Digest();
-        r = md.digest(Long.toString(key).getBytes(UTF_8));
-        cache.putIfAbsent(key, r);
-        return r;
-    }
-
-    @Override
-    public boolean isDeterministic()
-    {
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/Distribution.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/Distribution.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/Distribution.java
deleted file mode 100644
index ed40290..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/Distribution.java
+++ /dev/null
@@ -1,40 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-public abstract class Distribution
-{
-
-    public abstract long next();
-    public abstract long inverseCumProb(double cumProb);
-
-    public long maxValue()
-    {
-        return inverseCumProb(1d);
-    }
-
-    public long minValue()
-    {
-        return inverseCumProb(0d);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionBoundApache.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionBoundApache.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionBoundApache.java
deleted file mode 100644
index 0fdd437..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionBoundApache.java
+++ /dev/null
@@ -1,63 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import org.apache.commons.math3.distribution.AbstractRealDistribution;
-
-public class DistributionBoundApache extends Distribution
-{
-
-    final AbstractRealDistribution delegate;
-    final long min, max;
-
-    public DistributionBoundApache(AbstractRealDistribution delegate, long min, long max)
-    {
-        this.delegate = delegate;
-        this.min = min;
-        this.max = max;
-    }
-
-    @Override
-    public long next()
-    {
-        return bound(min, max, delegate.sample());
-    }
-
-    @Override
-    public long inverseCumProb(double cumProb)
-    {
-        return bound(min, max, delegate.inverseCumulativeProbability(cumProb));
-    }
-
-    private static long bound(long min, long max, double val)
-    {
-        long r = (long) val;
-        if ((r >= min) & (r <= max))
-            return r;
-        if (r < min)
-            return min;
-        if (r > max)
-            return max;
-        throw new IllegalStateException();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFactory.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFactory.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFactory.java
deleted file mode 100644
index b80f851..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFactory.java
+++ /dev/null
@@ -1,31 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-import java.io.Serializable;
-
-public interface DistributionFactory extends Serializable
-{
-
-    Distribution get();
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFixed.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFixed.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFixed.java
deleted file mode 100644
index 43372ca..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFixed.java
+++ /dev/null
@@ -1,46 +0,0 @@
-package org.apache.cassandra.stress.generatedata;
-/*
- * 
- * 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.
- * 
- */
-
-
-public class DistributionFixed extends Distribution
-{
-
-    final long key;
-
-    public DistributionFixed(long key)
-    {
-        this.key = key;
-    }
-
-    @Override
-    public long next()
-    {
-        return key;
-    }
-
-    @Override
-    public long inverseCumProb(double cumProb)
-    {
-        return key;
-    }
-
-}