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:39 UTC

[16/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/settings/OptionAnyProbabilities.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionAnyProbabilities.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionAnyProbabilities.java
new file mode 100644
index 0000000..a9af1bd
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionAnyProbabilities.java
@@ -0,0 +1,78 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.math3.util.Pair;
+
+public final class OptionAnyProbabilities extends OptionMulti
+{
+    public OptionAnyProbabilities(String name, String description)
+    {
+        super(name, description, false);
+    }
+
+    final CollectRatios ratios = new CollectRatios();
+
+    private static final class CollectRatios extends Option
+    {
+        Map<String, Double> options = new LinkedHashMap<>();
+
+        boolean accept(String param)
+        {
+            String[] args = param.split("=");
+            if (args.length == 2 && args[1].length() > 0 && args[0].length() > 0)
+            {
+                if (options.put(args[0], Double.parseDouble(args[1])) != null)
+                    throw new IllegalArgumentException(args[0] + " set twice");
+                return true;
+            }
+            return false;
+        }
+
+        boolean happy()
+        {
+            return !options.isEmpty();
+        }
+
+        String shortDisplay()
+        {
+            return null;
+        }
+
+        String longDisplay()
+        {
+            return null;
+        }
+
+        List<String> multiLineDisplay()
+        {
+            return Collections.emptyList();
+        }
+
+        boolean setByUser()
+        {
+            return !options.isEmpty();
+        }
+    }
+
+
+    @Override
+    public List<? extends Option> options()
+    {
+        return Arrays.asList(ratios);
+    }
+
+    List<Pair<String, Double>> ratios()
+    {
+        List<Pair<String, Double>> ratiosOut = new ArrayList<>();
+        for (Map.Entry<String, Double> e : ratios.options.entrySet())
+            ratiosOut.add(new Pair<String, Double>(e.getKey(), e.getValue()));
+        return ratiosOut;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/OptionDataGen.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionDataGen.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionDataGen.java
deleted file mode 100644
index bde2b10..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionDataGen.java
+++ /dev/null
@@ -1,203 +0,0 @@
-package org.apache.cassandra.stress.settings;
-/*
- * 
- * 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.File;
-import java.util.*;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.cassandra.stress.generatedata.*;
-
-/**
- * For selecting a data generator
- */
-class OptionDataGen extends Option
-{
-
-    private static final Pattern FULL = Pattern.compile("([A-Z]+)\\(([^)]+)\\)", Pattern.CASE_INSENSITIVE);
-    private static final Pattern ARGS = Pattern.compile("[^,]+");
-
-    final String prefix;
-    private DataGenFactory factory;
-    private final DataGenFactory defaultFactory;
-
-    public OptionDataGen(String prefix, String defaultSpec)
-    {
-        this.prefix = prefix;
-        this.defaultFactory = defaultSpec == null ? null : get(defaultSpec);
-    }
-
-    @Override
-    public boolean accept(String param)
-    {
-        if (!param.toLowerCase().startsWith(prefix))
-            return false;
-        factory = get(param.substring(prefix.length()));
-        return true;
-    }
-
-    private static DataGenFactory get(String spec)
-    {
-        Matcher m = FULL.matcher(spec);
-        if (!m.matches())
-            throw new IllegalArgumentException("Illegal data generator specification: " + spec);
-        String name = m.group(1);
-        Impl impl = LOOKUP.get(name.toLowerCase());
-        if (impl == null)
-            throw new IllegalArgumentException("Illegal data generator type: " + name);
-        List<String> params = new ArrayList<>();
-        m = ARGS.matcher(m.group(2));
-        while (m.find())
-            params.add(m.group());
-        return impl.getFactory(params);
-    }
-
-    public DataGenFactory get()
-    {
-        return factory != null ? factory : defaultFactory;
-    }
-
-    @Override
-    public boolean happy()
-    {
-        return factory != null || defaultFactory != null;
-    }
-
-    public boolean setByUser()
-    {
-        return factory != null;
-    }
-
-    @Override
-    public String shortDisplay()
-    {
-        return prefix + "ALG()";
-    }
-
-    public String longDisplay()
-    {
-        return shortDisplay() + ": Specify a data generator from:";
-    }
-
-    @Override
-    public List<String> multiLineDisplay()
-    {
-        return Arrays.asList(
-                GroupedOptions.formatMultiLine("RANDOM()", "Completely random byte generation"),
-                GroupedOptions.formatMultiLine("REPEAT(<freq>)", "An MD5 hash of (opIndex % freq) combined with the column index"),
-                GroupedOptions.formatMultiLine("DICT(<file>)","Random words from a dictionary; the file should be in the format \"<freq> <word>\"")
-        );
-    }
-
-    private static final Map<String, Impl> LOOKUP;
-    static
-    {
-        final Map<String, Impl> lookup = new HashMap<>();
-        lookup.put("random", new RandomImpl());
-        lookup.put("rand", new RandomImpl());
-        lookup.put("rnd", new RandomImpl());
-        lookup.put("repeat", new RepeatImpl());
-        lookup.put("dict", new DictionaryImpl());
-        lookup.put("dictionary", new DictionaryImpl());
-        LOOKUP = lookup;
-    }
-
-    private static interface Impl
-    {
-        public DataGenFactory getFactory(List<String> params);
-    }
-
-    private static final class RandomImpl implements Impl
-    {
-        @Override
-        public DataGenFactory getFactory(List<String> params)
-        {
-            if (params.size() != 0)
-                throw new IllegalArgumentException("Invalid parameter list for random generator: " + params);
-            return new RandomFactory();
-        }
-    }
-
-    private static final class RepeatImpl implements Impl
-    {
-
-        @Override
-        public DataGenFactory getFactory(List<String> params)
-        {
-            if (params.size() != 1)
-                throw new IllegalArgumentException("Invalid parameter list for repeating generator: " + params);
-            try
-            {
-                int repeatFrequency = Integer.parseInt(params.get(0));
-                return new RepeatsFactory(repeatFrequency);
-            } catch (Exception _)
-            {
-                throw new IllegalArgumentException("Invalid parameter list for repeating generator: " + params);
-            }
-        }
-    }
-
-    private static final class DictionaryImpl implements Impl
-    {
-
-        @Override
-        public DataGenFactory getFactory(List<String> params)
-        {
-            if (params.size() != 1)
-                throw new IllegalArgumentException("Invalid parameter list for dictionary generator: " + params);
-            try
-            {
-                final File file = new File(params.get(0));
-                return DataGenStringDictionary.getFactory(file);
-            } catch (Exception e)
-            {
-                throw new IllegalArgumentException("Invalid parameter list for dictionary generator: " + params, e);
-            }
-        }
-    }
-
-    private static final class RandomFactory implements DataGenFactory
-    {
-        @Override
-        public DataGen get()
-        {
-            return new DataGenBytesRandom();
-        }
-    }
-
-    private static final class RepeatsFactory implements DataGenFactory
-    {
-        final int frequency;
-        private RepeatsFactory(int frequency)
-        {
-            this.frequency = frequency;
-        }
-
-        @Override
-        public DataGen get()
-        {
-            return new DataGenStringRepeats(frequency);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java
index 76fa0a9..70a85ae 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java
@@ -25,19 +25,20 @@ import java.util.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.cassandra.stress.generatedata.*;
+import org.apache.cassandra.stress.generate.*;
 import org.apache.commons.math3.distribution.ExponentialDistribution;
 import org.apache.commons.math3.distribution.NormalDistribution;
 import org.apache.commons.math3.distribution.UniformRealDistribution;
 import org.apache.commons.math3.distribution.WeibullDistribution;
+import org.apache.commons.math3.random.JDKRandomGenerator;
 
 /**
  * For selecting a mathematical distribution
  */
-class OptionDistribution extends Option
+public class OptionDistribution extends Option
 {
 
-    private static final Pattern FULL = Pattern.compile("([A-Z]+)\\((.+)\\)", Pattern.CASE_INSENSITIVE);
+    private static final Pattern FULL = Pattern.compile("(~?)([A-Z]+)\\((.+)\\)", Pattern.CASE_INSENSITIVE);
     private static final Pattern ARGS = Pattern.compile("[^,]+");
 
     final String prefix;
@@ -61,20 +62,22 @@ class OptionDistribution extends Option
         return true;
     }
 
-    private static DistributionFactory get(String spec)
+    public static DistributionFactory get(String spec)
     {
         Matcher m = FULL.matcher(spec);
         if (!m.matches())
             throw new IllegalArgumentException("Illegal distribution specification: " + spec);
-        String name = m.group(1);
+        boolean inverse = m.group(1).equals("~");
+        String name = m.group(2);
         Impl impl = LOOKUP.get(name.toLowerCase());
         if (impl == null)
             throw new IllegalArgumentException("Illegal distribution type: " + name);
         List<String> params = new ArrayList<>();
-        m = ARGS.matcher(m.group(2));
+        m = ARGS.matcher(m.group(3));
         while (m.find())
             params.add(m.group());
-        return impl.getFactory(params);
+        DistributionFactory factory = impl.getFactory(params);
+        return inverse ? new InverseFactory(factory) : factory;
     }
 
     public DistributionFactory get()
@@ -103,6 +106,7 @@ class OptionDistribution extends Option
                 GroupedOptions.formatMultiLine("GAUSSIAN(min..max,mean,stdev)", "A gaussian/normal distribution, with explicitly defined mean and stdev"),
                 GroupedOptions.formatMultiLine("UNIFORM(min..max)", "A uniform distribution over the range [min, max]"),
                 GroupedOptions.formatMultiLine("FIXED(val)", "A fixed distribution, always returning the same value"),
+                "Preceding the name with ~ will invert the distribution, e.g. ~exp(1..10) will yield 10 most, instead of least, often",
                 "Aliases: extr, gauss, normal, norm, weibull"
         );
     }
@@ -142,6 +146,23 @@ class OptionDistribution extends Option
         public DistributionFactory getFactory(List<String> params);
     }
 
+    public static long parseLong(String value)
+    {
+        long multiplier = 1;
+        value = value.trim().toLowerCase();
+        switch (value.charAt(value.length() - 1))
+        {
+            case 'b':
+                multiplier *= 1000;
+            case 'm':
+                multiplier *= 1000;
+            case 'k':
+                multiplier *= 1000;
+                value = value.substring(0, value.length() - 1);
+        }
+        return Long.parseLong(value) * multiplier;
+    }
+
     private static final class GaussianImpl implements Impl
     {
 
@@ -153,8 +174,8 @@ class OptionDistribution extends Option
             try
             {
                 String[] bounds = params.get(0).split("\\.\\.+");
-                final long min = Long.parseLong(bounds[0]);
-                final long max = Long.parseLong(bounds[1]);
+                final long min = parseLong(bounds[0]);
+                final long max = parseLong(bounds[1]);
                 final double mean, stdev;
                 if (params.size() == 3)
                 {
@@ -185,8 +206,8 @@ class OptionDistribution extends Option
             try
             {
                 String[] bounds = params.get(0).split("\\.\\.+");
-                final long min = Long.parseLong(bounds[0]);
-                final long max = Long.parseLong(bounds[1]);
+                final long min = parseLong(bounds[0]);
+                final long max = parseLong(bounds[1]);
                 ExponentialDistribution findBounds = new ExponentialDistribution(1d);
                 // max probability should be roughly equal to accuracy of (max-min) to ensure all values are visitable,
                 // over entire range, but this results in overly skewed distribution, so take sqrt
@@ -209,8 +230,8 @@ class OptionDistribution extends Option
             try
             {
                 String[] bounds = params.get(0).split("\\.\\.+");
-                final long min = Long.parseLong(bounds[0]);
-                final long max = Long.parseLong(bounds[1]);
+                final long min = parseLong(bounds[0]);
+                final long max = parseLong(bounds[1]);
                 final double shape = Double.parseDouble(params.get(1));
                 WeibullDistribution findBounds = new WeibullDistribution(shape, 1d);
                 // max probability should be roughly equal to accuracy of (max-min) to ensure all values are visitable,
@@ -235,8 +256,8 @@ class OptionDistribution extends Option
             try
             {
                 String[] bounds = params.get(0).split("\\.\\.+");
-                final long min = Long.parseLong(bounds[0]);
-                final long max = Long.parseLong(bounds[1]);
+                final long min = parseLong(bounds[0]);
+                final long max = parseLong(bounds[1]);
                 return new UniformFactory(min, max);
             } catch (Exception _)
             {
@@ -255,7 +276,7 @@ class OptionDistribution extends Option
                 throw new IllegalArgumentException("Invalid parameter list for uniform distribution: " + params);
             try
             {
-                final long key = Long.parseLong(params.get(0));
+                final long key = parseLong(params.get(0));
                 return new FixedFactory(key);
             } catch (Exception _)
             {
@@ -264,6 +285,20 @@ class OptionDistribution extends Option
         }
     }
 
+    private static final class InverseFactory implements DistributionFactory
+    {
+        final DistributionFactory wrapped;
+        private InverseFactory(DistributionFactory wrapped)
+        {
+            this.wrapped = wrapped;
+        }
+
+        public Distribution get()
+        {
+            return new DistributionInverted(wrapped.get());
+        }
+    }
+
     // factories
 
     private static final class ExpFactory implements DistributionFactory
@@ -280,7 +315,7 @@ class OptionDistribution extends Option
         @Override
         public Distribution get()
         {
-            return new DistributionOffsetApache(new ExponentialDistribution(mean), min, max);
+            return new DistributionOffsetApache(new ExponentialDistribution(new JDKRandomGenerator(), mean, ExponentialDistribution.DEFAULT_INVERSE_ABSOLUTE_ACCURACY), min, max);
         }
     }
 
@@ -299,7 +334,7 @@ class OptionDistribution extends Option
         @Override
         public Distribution get()
         {
-            return new DistributionOffsetApache(new WeibullDistribution(shape, scale), min, max);
+            return new DistributionOffsetApache(new WeibullDistribution(new JDKRandomGenerator(), shape, scale, WeibullDistribution.DEFAULT_INVERSE_ABSOLUTE_ACCURACY), min, max);
         }
     }
 
@@ -318,7 +353,7 @@ class OptionDistribution extends Option
         @Override
         public Distribution get()
         {
-            return new DistributionBoundApache(new NormalDistribution(mean, stdev), min, max);
+            return new DistributionBoundApache(new NormalDistribution(new JDKRandomGenerator(), mean, stdev, NormalDistribution.DEFAULT_INVERSE_ABSOLUTE_ACCURACY), min, max);
         }
     }
 
@@ -334,7 +369,7 @@ class OptionDistribution extends Option
         @Override
         public Distribution get()
         {
-            return new DistributionBoundApache(new UniformRealDistribution(min, max + 1), min, max);
+            return new DistributionBoundApache(new UniformRealDistribution(new JDKRandomGenerator(), min, max + 1), min, max);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/OptionEnumProbabilities.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionEnumProbabilities.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionEnumProbabilities.java
new file mode 100644
index 0000000..88ebd34
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionEnumProbabilities.java
@@ -0,0 +1,62 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.math3.util.Pair;
+
+public final class OptionEnumProbabilities<T> extends OptionMulti
+{
+    final List<OptMatcher<T>> options;
+
+    public static class Opt<T>
+    {
+        final T option;
+        final String defaultValue;
+
+        public Opt(T option, String defaultValue)
+        {
+            this.option = option;
+            this.defaultValue = defaultValue;
+        }
+    }
+
+    private static final class OptMatcher<T> extends OptionSimple
+    {
+        final T opt;
+        OptMatcher(T opt, String defaultValue)
+        {
+            super(opt.toString().toLowerCase() + "=", "[0-9]+(\\.[0-9]+)?", defaultValue, "Performs this many " + opt + " operations out of total", false);
+            this.opt = opt;
+        }
+    }
+
+    public OptionEnumProbabilities(List<Opt<T>> universe, String name, String description)
+    {
+        super(name, description, false);
+        List<OptMatcher<T>> options = new ArrayList<>();
+        for (Opt<T> option : universe)
+            options.add(new OptMatcher<T>(option.option, option.defaultValue));
+        this.options = options;
+    }
+
+    @Override
+    public List<? extends Option> options()
+    {
+        return options;
+    }
+
+    List<Pair<T, Double>> ratios()
+    {
+        List<? extends Option> ratiosIn = setByUser() ? optionsSetByUser() : defaultOptions();
+        List<Pair<T, Double>> ratiosOut = new ArrayList<>();
+        for (Option opt : ratiosIn)
+        {
+            OptMatcher<T> optMatcher = (OptMatcher<T>) opt;
+            double d = Double.parseDouble(optMatcher.value());
+            ratiosOut.add(new Pair<>(optMatcher.opt, d));
+        }
+        return ratiosOut;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/OptionMulti.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionMulti.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionMulti.java
index 60faad8..32bfc65 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionMulti.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionMulti.java
@@ -156,7 +156,8 @@ abstract class OptionMulti extends Option
             String[] args = param.split("=");
             if (args.length == 2 && args[1].length() > 0 && args[0].length() > 0)
             {
-                options.put(args[0], args[1]);
+                if (options.put(args[0], args[1]) != null)
+                    throw new IllegalArgumentException(args[0] + " set twice");
                 return true;
             }
             return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/OptionRatioDistribution.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionRatioDistribution.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionRatioDistribution.java
new file mode 100644
index 0000000..2459c20
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionRatioDistribution.java
@@ -0,0 +1,166 @@
+package org.apache.cassandra.stress.settings;
+/*
+ * 
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.math3.distribution.ExponentialDistribution;
+import org.apache.commons.math3.distribution.NormalDistribution;
+import org.apache.commons.math3.distribution.UniformRealDistribution;
+import org.apache.commons.math3.distribution.WeibullDistribution;
+import org.apache.commons.math3.random.JDKRandomGenerator;
+
+import org.apache.cassandra.stress.generate.Distribution;
+import org.apache.cassandra.stress.generate.DistributionBoundApache;
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.stress.generate.DistributionFixed;
+import org.apache.cassandra.stress.generate.DistributionInverted;
+import org.apache.cassandra.stress.generate.DistributionOffsetApache;
+import org.apache.cassandra.stress.generate.RatioDistribution;
+import org.apache.cassandra.stress.generate.RatioDistributionFactory;
+
+/**
+ * For selecting a mathematical distribution
+ */
+public class OptionRatioDistribution extends Option
+{
+
+    private static final Pattern FULL = Pattern.compile("(.*)/([0-9]+[KMB]?)", Pattern.CASE_INSENSITIVE);
+
+    final OptionDistribution delegate;
+    private double divisor;
+
+    private static final RatioDistribution DEFAULT = new RatioDistribution(new DistributionFixed(1), 1);
+
+    public OptionRatioDistribution(String prefix, String defaultSpec, String description)
+    {
+        delegate = new OptionDistribution(prefix, defaultSpec, description);
+    }
+
+    @Override
+    public boolean accept(String param)
+    {
+        Matcher m = FULL.matcher(param);
+        if (!m.matches() || !delegate.accept(m.group(1)))
+            return false;
+        divisor = OptionDistribution.parseLong(m.group(2));
+        return true;
+    }
+
+    public static RatioDistributionFactory get(String spec)
+    {
+        OptionRatioDistribution opt = new OptionRatioDistribution("", "", "");
+        if (!opt.accept(spec))
+            throw new IllegalArgumentException();
+        return opt.get();
+    }
+
+    public RatioDistributionFactory get()
+    {
+        return !delegate.setByUser() ? new DefaultFactory() : new DelegateFactory(delegate.get(), divisor);
+    }
+
+    @Override
+    public boolean happy()
+    {
+        return delegate.happy();
+    }
+
+    public String longDisplay()
+    {
+        return delegate.longDisplay();
+    }
+
+    @Override
+    public List<String> multiLineDisplay()
+    {
+        return Arrays.asList(
+                GroupedOptions.formatMultiLine("EXP(min..max)/divisor", "An exponential ratio distribution over the range [min..max]/divisor"),
+                GroupedOptions.formatMultiLine("EXTREME(min..max,shape)/divisor", "An extreme value (Weibull) ratio distribution over the range [min..max]/divisor"),
+                GroupedOptions.formatMultiLine("GAUSSIAN(min..max,stdvrng)/divisor", "A gaussian/normal ratio distribution, where mean=(min+max)/2, and stdev is ((mean-min)/stdvrng)/divisor"),
+                GroupedOptions.formatMultiLine("GAUSSIAN(min..max,mean,stdev)/divisor", "A gaussian/normal ratio distribution, with explicitly defined mean and stdev"),
+                GroupedOptions.formatMultiLine("UNIFORM(min..max)/divisor", "A uniform ratio distribution over the range [min, max]/divisor"),
+                GroupedOptions.formatMultiLine("FIXED(val)/divisor", "A fixed ratio distribution, always returning the same value"),
+                "Preceding the name with ~ will invert the distribution, e.g. ~exp(1..10)/10 will yield 0.1 least, instead of most, often",
+                "Aliases: extr, gauss, normal, norm, weibull"
+        );
+    }
+
+    boolean setByUser()
+    {
+        return delegate.setByUser();
+    }
+
+    @Override
+    public String shortDisplay()
+    {
+        return delegate.shortDisplay();
+    }
+
+    // factories
+
+    private static final class DefaultFactory implements RatioDistributionFactory
+    {
+        @Override
+        public RatioDistribution get()
+        {
+            return DEFAULT;
+        }
+    }
+
+    private static final class DelegateFactory implements RatioDistributionFactory
+    {
+        final DistributionFactory delegate;
+        final double divisor;
+
+        private DelegateFactory(DistributionFactory delegate, double divisor)
+        {
+            this.delegate = delegate;
+            this.divisor = divisor;
+        }
+
+        @Override
+        public RatioDistribution get()
+        {
+            return new RatioDistribution(delegate.get(), divisor);
+        }
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return delegate.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return super.equals(that) && ((OptionRatioDistribution) that).delegate.equals(this.delegate);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/OptionSimple.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionSimple.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionSimple.java
index 9365e45..71c1ffe 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionSimple.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionSimple.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.stress.settings;
  */
 
 
+import java.io.Serializable;
 import java.util.Collections;
 import java.util.List;
 import java.util.regex.Pattern;
@@ -30,7 +31,7 @@ import com.google.common.base.Function;
 /**
  * For parsing a simple (sub)option for a command/major option
  */
-class OptionSimple extends Option
+class OptionSimple extends Option implements Serializable
 {
 
     final String displayPrefix;
@@ -41,7 +42,7 @@ class OptionSimple extends Option
     private final boolean required;
     private String value;
 
-    private static final class ValueMatcher implements Function<String, String>
+    private static final class ValueMatcher implements Function<String, String>, Serializable
     {
         final Pattern pattern;
         private ValueMatcher(Pattern pattern)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsColumn.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsColumn.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsColumn.java
index 7e20ec6..04c2a47 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsColumn.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsColumn.java
@@ -23,13 +23,15 @@ package org.apache.cassandra.stress.settings;
 
 import java.io.Serializable;
 import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.stress.generatedata.*;
+import org.apache.cassandra.stress.generate.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -39,17 +41,13 @@ public class SettingsColumn implements Serializable
 {
 
     public final int maxColumnsPerKey;
-    public final List<ByteBuffer> names;
+    public transient final List<ByteBuffer> names;
     public final List<String> namestrs;
     public final String comparator;
-    public final boolean useTimeUUIDComparator;
-    public final int superColumns;
-    public final boolean useSuperColumns;
     public final boolean variableColumnCount;
     public final boolean slice;
-    private final DistributionFactory sizeDistribution;
-    private final DistributionFactory countDistribution;
-    private final DataGenFactory dataGenFactory;
+    public final DistributionFactory sizeDistribution;
+    public final DistributionFactory countDistribution;
 
     public SettingsColumn(GroupedOptions options)
     {
@@ -62,9 +60,6 @@ public class SettingsColumn implements Serializable
     public SettingsColumn(Options options, NameOptions name, CountOptions count)
     {
         sizeDistribution = options.size.get();
-        superColumns = Integer.parseInt(options.superColumns.value());
-        dataGenFactory = options.generator.get();
-        useSuperColumns = superColumns > 0;
         {
             comparator = options.comparator.value();
             AbstractType parsed = null;
@@ -79,8 +74,6 @@ public class SettingsColumn implements Serializable
                 System.exit(1);
             }
 
-            useTimeUUIDComparator = parsed instanceof TimeUUIDType;
-
             if (!(parsed instanceof TimeUUIDType || parsed instanceof AsciiType || parsed instanceof UTF8Type))
             {
                 System.err.println("Currently supported types are: TimeUUIDType, AsciiType, UTF8Type.");
@@ -102,10 +95,13 @@ public class SettingsColumn implements Serializable
 
             final String[] names = name.name.value().split(",");
             this.names = new ArrayList<>(names.length);
-            this.namestrs = Arrays.asList(names);
 
             for (String columnName : names)
                 this.names.add(comparator.fromString(columnName));
+            Collections.sort(this.names, BytesType.instance);
+            this.namestrs = new ArrayList<>();
+            for (ByteBuffer columnName : this.names)
+                this.namestrs.add(comparator.getString(columnName));
 
             final int nameCount = this.names.size();
             countDistribution = new DistributionFactory()
@@ -123,23 +119,24 @@ public class SettingsColumn implements Serializable
             ByteBuffer[] names = new ByteBuffer[(int) countDistribution.get().maxValue()];
             String[] namestrs = new String[(int) countDistribution.get().maxValue()];
             for (int i = 0 ; i < names.length ; i++)
-            {
                 names[i] = ByteBufferUtil.bytes("C" + i);
-                namestrs[i] = "C" + i;
+            Arrays.sort(names, BytesType.instance);
+            try
+            {
+                for (int i = 0 ; i < names.length ; i++)
+                    namestrs[i] = ByteBufferUtil.string(names[i]);
+            }
+            catch (CharacterCodingException e)
+            {
+                throw new RuntimeException(e);
             }
+
             this.names = Arrays.asList(names);
             this.namestrs = Arrays.asList(namestrs);
         }
         maxColumnsPerKey = (int) countDistribution.get().maxValue();
         variableColumnCount = countDistribution.get().minValue() < maxColumnsPerKey;
-        // TODO: should warn that we always slice for useTimeUUIDComparator?
-        slice = options.slice.setByUser() || useTimeUUIDComparator;
-        // TODO: with useTimeUUIDCOmparator, should we still try to select a random start for reads if possible?
-    }
-
-    public RowGen newRowGen()
-    {
-        return new RowGenDistributedSize(dataGenFactory.get(), countDistribution.get(), sizeDistribution.get());
+        slice = options.slice.setByUser();
     }
 
     // Option Declarations
@@ -150,7 +147,6 @@ public class SettingsColumn implements Serializable
         final OptionSimple comparator = new OptionSimple("comparator=", "TimeUUIDType|AsciiType|UTF8Type", "AsciiType", "Column Comparator to use", false);
         final OptionSimple slice = new OptionSimple("slice", "", null, "If set, range slices will be used for reads, otherwise a names query will be", false);
         final OptionDistribution size = new OptionDistribution("size=", "FIXED(34)", "Cell size distribution");
-        final OptionDataGen generator = new OptionDataGen("data=", "REPEAT(50)");
     }
 
     private static final class NameOptions extends Options
@@ -160,7 +156,7 @@ public class SettingsColumn implements Serializable
         @Override
         public List<? extends Option> options()
         {
-            return Arrays.asList(name, slice, superColumns, comparator, size, generator);
+            return Arrays.asList(name, slice, superColumns, comparator, size);
         }
     }
 
@@ -171,7 +167,7 @@ public class SettingsColumn implements Serializable
         @Override
         public List<? extends Option> options()
         {
-            return Arrays.asList(count, slice, superColumns, comparator, size, generator);
+            return Arrays.asList(count, slice, superColumns, comparator, size);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
index b491707..032f00c 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
@@ -26,23 +26,24 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.stress.generatedata.DistributionFactory;
+import org.apache.cassandra.stress.operations.OpDistributionFactory;
 import org.apache.cassandra.thrift.ConsistencyLevel;
 
 // Generic command settings - common to read/write/etc
-public class SettingsCommand implements Serializable
+public abstract class SettingsCommand implements Serializable
 {
 
     public final Command type;
     public final long count;
     public final int tries;
     public final boolean ignoreErrors;
+    public final boolean noWarmup;
     public final ConsistencyLevel consistencyLevel;
     public final double targetUncertainty;
     public final int minimumUncertaintyMeasurements;
     public final int maximumUncertaintyMeasurements;
-    public final DistributionFactory add;
-    public final int keysAtOnce;
+
+    public abstract OpDistributionFactory getFactory(StressSettings settings);
 
     public SettingsCommand(Command type, GroupedOptions options)
     {
@@ -58,8 +59,7 @@ public class SettingsCommand implements Serializable
         this.tries = Math.max(1, Integer.parseInt(options.retries.value()) + 1);
         this.ignoreErrors = options.ignoreErrors.setByUser();
         this.consistencyLevel = ConsistencyLevel.valueOf(options.consistencyLevel.value().toUpperCase());
-        this.keysAtOnce = Integer.parseInt(options.atOnce.value());
-        this.add = options.add.get();
+        this.noWarmup = options.noWarmup.setByUser();
         if (count != null)
         {
             this.count = Long.parseLong(count.count.value());
@@ -82,8 +82,8 @@ public class SettingsCommand implements Serializable
     {
         final OptionSimple retries = new OptionSimple("tries=", "[0-9]+", "9", "Number of tries to perform for each operation before failing", false);
         final OptionSimple ignoreErrors = new OptionSimple("ignore_errors", "", null, "Do not print/log errors", false);
+        final OptionSimple noWarmup = new OptionSimple("no_warmup", "", null, "Do not warmup the process", false);
         final OptionSimple consistencyLevel = new OptionSimple("cl=", "ONE|QUORUM|LOCAL_QUORUM|EACH_QUORUM|ALL|ANY", "ONE", "Consistency level to use", false);
-        final OptionDistribution add = new OptionDistribution("add=", "fixed(1)", "Distribution of value of counter increments");
         final OptionSimple atOnce = new OptionSimple("at-once=", "[0-9]+", "1000", "Number of keys per operation for multiget", false);
     }
 
@@ -93,7 +93,7 @@ public class SettingsCommand implements Serializable
         @Override
         public List<? extends Option> options()
         {
-            return Arrays.asList(count, retries, ignoreErrors, consistencyLevel, add, atOnce);
+            return Arrays.asList(count, retries, noWarmup, ignoreErrors, consistencyLevel, atOnce);
         }
     }
 
@@ -105,7 +105,7 @@ public class SettingsCommand implements Serializable
         @Override
         public List<? extends Option> options()
         {
-            return Arrays.asList(uncertainty, minMeasurements, maxMeasurements, retries, ignoreErrors, consistencyLevel, add, atOnce);
+            return Arrays.asList(uncertainty, minMeasurements, maxMeasurements, retries, noWarmup, ignoreErrors, consistencyLevel, atOnce);
         }
     }
 
@@ -127,48 +127,16 @@ public class SettingsCommand implements Serializable
                 switch (cmd.category)
                 {
                     case BASIC:
-                    case MULTI:
-                        return build(cmd, params);
+                        return SettingsCommandPreDefined.build(cmd, params);
                     case MIXED:
-                        return SettingsCommandMixed.build(params);
+                        return SettingsCommandPreDefinedMixed.build(params);
+                    case USER:
+                        return SettingsCommandUser.build(params);
                 }
             }
         }
         return null;
     }
 
-    static SettingsCommand build(Command type, String[] params)
-    {
-        GroupedOptions options = GroupedOptions.select(params, new Count(), new Uncertainty());
-        if (options == null)
-        {
-            printHelp(type);
-            System.out.println("Invalid " + type + " options provided, see output for valid options");
-            System.exit(1);
-        }
-        return new SettingsCommand(type, options);
-    }
-
-    static void printHelp(Command type)
-    {
-        printHelp(type.toString().toLowerCase());
-    }
-
-    static void printHelp(String type)
-    {
-        GroupedOptions.printOptions(System.out, type.toLowerCase(), new Uncertainty(), new Count());
-    }
-
-    static Runnable helpPrinter(final Command type)
-    {
-        return new Runnable()
-        {
-            @Override
-            public void run()
-            {
-                printHelp(type);
-            }
-        };
-    }
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMixed.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMixed.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMixed.java
deleted file mode 100644
index 3f16425..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMixed.java
+++ /dev/null
@@ -1,207 +0,0 @@
-package org.apache.cassandra.stress.settings;
-/*
- * 
- * 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.util.ArrayList;
-import java.util.List;
-
-import org.apache.cassandra.stress.generatedata.Distribution;
-import org.apache.cassandra.stress.generatedata.DistributionFactory;
-
-import org.apache.commons.math3.distribution.EnumeratedDistribution;
-import org.apache.commons.math3.util.Pair;
-
-// Settings unique to the mixed command type
-public class SettingsCommandMixed extends SettingsCommand
-{
-
-    // Ratios for selecting commands - index for each Command, NaN indicates the command is not requested
-    private final List<Pair<Command, Double>> ratios;
-    private final DistributionFactory clustering;
-
-    public SettingsCommandMixed(Options options)
-    {
-        super(Command.MIXED, options.parent);
-
-        clustering = options.clustering.get();
-        ratios = options.probabilities.ratios();
-        if (ratios.size() == 0)
-            throw new IllegalArgumentException("Must specify at least one command with a non-zero ratio");
-    }
-
-    public List<Command> getCommands()
-    {
-        final List<Command> r = new ArrayList<>();
-        for (Pair<Command, Double> p : ratios)
-            r.add(p.getFirst());
-        return r;
-    }
-
-    public CommandSelector selector()
-    {
-        return new CommandSelector(ratios, clustering.get());
-    }
-
-    // Class for randomly selecting the next command type
-
-    public static final class CommandSelector
-    {
-
-        final EnumeratedDistribution<Command> selector;
-        final Distribution count;
-        private Command cur;
-        private long remaining;
-
-        public CommandSelector(List<Pair<Command, Double>> ratios, Distribution count)
-        {
-            selector = new EnumeratedDistribution<>(ratios);
-            this.count = count;
-        }
-
-        public Command next()
-        {
-            while (remaining == 0)
-            {
-                remaining = count.next();
-                cur = selector.sample();
-            }
-            remaining--;
-            return cur;
-        }
-    }
-
-    // Option Declarations
-
-    static final class Probabilities extends OptionMulti
-    {
-        // entry for each in Command.values()
-        final OptionSimple[] ratios;
-        final List<OptionSimple> grouping;
-
-        public Probabilities()
-        {
-            super("ratio", "Specify the ratios for operations to perform; e.g. (reads=2,writes=1) will perform 2 reads for each write", false);
-            OptionSimple[] ratios = new OptionSimple[Command.values().length];
-            List<OptionSimple> grouping = new ArrayList<>();
-            for (Command command : Command.values())
-            {
-                if (command.category == null)
-                    continue;
-                String defaultValue;
-                switch (command)
-                {
-                    case MIXED:
-                        continue;
-                    case READ:
-                    case WRITE:
-                        defaultValue = "1";
-                        break;
-                    default:
-                        defaultValue = null;
-                }
-                OptionSimple ratio = new OptionSimple(command.toString().toLowerCase() +
-                        "=", "[0-9]+(\\.[0-9]+)?", defaultValue, "Performs this many " + command + " operations out of total", false);
-                ratios[command.ordinal()] = ratio;
-                grouping.add(ratio);
-            }
-            this.grouping = grouping;
-            this.ratios = ratios;
-        }
-
-        @Override
-        public List<? extends Option> options()
-        {
-            return grouping;
-        }
-
-        List<Pair<Command, Double>> ratios()
-        {
-            List<? extends Option> ratiosIn = setByUser() ? optionsSetByUser() : defaultOptions();
-            List<Pair<Command, Double>> ratiosOut = new ArrayList<>();
-            for (Option opt : ratiosIn)
-            {
-                OptionSimple ratioIn = (OptionSimple) opt;
-                Command command = Command.get(ratioIn.displayPrefix.substring(0, ratioIn.displayPrefix.length() - 1));
-                double d = Double.parseDouble(ratioIn.value());
-                ratiosOut.add(new Pair<>(command, d));
-            }
-            return ratiosOut;
-        }
-    }
-
-    static final class Options extends GroupedOptions
-    {
-        final SettingsCommand.Options parent;
-        protected Options(SettingsCommand.Options parent)
-        {
-            this.parent = parent;
-        }
-        final OptionDistribution clustering = new OptionDistribution("clustering=", "GAUSSIAN(1..10)", "Distribution clustering runs of operations of the same kind");
-        final Probabilities probabilities = new Probabilities();
-
-        @Override
-        public List<? extends Option> options()
-        {
-            final List<Option> options = new ArrayList<>();
-            options.add(clustering);
-            options.add(probabilities);
-            options.addAll(parent.options());
-            return options;
-        }
-
-    }
-
-    // CLI utility methods
-
-    public static SettingsCommandMixed build(String[] params)
-    {
-        GroupedOptions options = GroupedOptions.select(params,
-                new Options(new SettingsCommand.Uncertainty()),
-                new Options(new SettingsCommand.Count()));
-        if (options == null)
-        {
-            printHelp();
-            System.out.println("Invalid MIXED options provided, see output for valid options");
-            System.exit(1);
-        }
-        return new SettingsCommandMixed((Options) options);
-    }
-
-    public static void printHelp()
-    {
-        GroupedOptions.printOptions(System.out, "mixed",
-                                    new Options(new SettingsCommand.Uncertainty()),
-                                    new Options(new SettingsCommand.Count()));
-    }
-
-    public static Runnable helpPrinter()
-    {
-        return new Runnable()
-        {
-            @Override
-            public void run()
-            {
-                printHelp();
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefined.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefined.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefined.java
new file mode 100644
index 0000000..ac113d1
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefined.java
@@ -0,0 +1,145 @@
+package org.apache.cassandra.stress.settings;
+/*
+ * 
+ * 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.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.stress.generate.PartitionGenerator;
+import org.apache.cassandra.stress.generate.values.Bytes;
+import org.apache.cassandra.stress.generate.values.Generator;
+import org.apache.cassandra.stress.generate.values.GeneratorConfig;
+import org.apache.cassandra.stress.generate.values.HexBytes;
+import org.apache.cassandra.stress.operations.FixedOpDistribution;
+import org.apache.cassandra.stress.operations.OpDistribution;
+import org.apache.cassandra.stress.operations.OpDistributionFactory;
+import org.apache.cassandra.stress.operations.predefined.PredefinedOperation;
+import org.apache.cassandra.stress.util.Timer;
+
+// Settings unique to the mixed command type
+public class SettingsCommandPreDefined extends SettingsCommand
+{
+
+    public final DistributionFactory add;
+
+    public OpDistributionFactory getFactory(final StressSettings settings)
+    {
+        return new OpDistributionFactory()
+        {
+            public OpDistribution get(Timer timer)
+            {
+                return new FixedOpDistribution(PredefinedOperation.operation(type, timer, newGenerator(settings), settings, add));
+            }
+
+            public String desc()
+            {
+                return type.toString();
+            }
+
+            public Iterable<OpDistributionFactory> each()
+            {
+                return Collections.<OpDistributionFactory>singleton(this);
+            }
+        };
+    }
+
+    PartitionGenerator newGenerator(StressSettings settings)
+    {
+        List<String> names = settings.columns.namestrs;
+        List<Generator> partitionKey = Collections.<Generator>singletonList(new HexBytes("key",
+                                       new GeneratorConfig("randomstrkey", null,
+                                                           OptionDistribution.get("fixed(" + settings.keys.keySize + ")"), null)));
+
+        List<Generator> columns = new ArrayList<>();
+        for (int i = 0 ; i < settings.columns.maxColumnsPerKey ; i++)
+            columns.add(new Bytes(names.get(i), new GeneratorConfig("randomstr" + names.get(i), null, settings.columns.sizeDistribution, null)));
+        return new PartitionGenerator(partitionKey, Collections.<Generator>emptyList(), columns);
+    }
+
+    public SettingsCommandPreDefined(Command type, Options options)
+    {
+        super(type, options.parent);
+        add = options.add.get();
+    }
+
+    // Option Declarations
+
+    static class Options extends GroupedOptions
+    {
+        final SettingsCommand.Options parent;
+        protected Options(SettingsCommand.Options parent)
+        {
+            this.parent = parent;
+        }
+        final OptionDistribution add = new OptionDistribution("add=", "fixed(1)", "Distribution of value of counter increments");
+
+        @Override
+        public List<? extends Option> options()
+        {
+            final List<Option> options = new ArrayList<>();
+            options.addAll(parent.options());
+            options.add(add);
+            return options;
+        }
+
+    }
+
+    // CLI utility methods
+
+    public static SettingsCommandPreDefined build(Command type, String[] params)
+    {
+        GroupedOptions options = GroupedOptions.select(params,
+                new Options(new Uncertainty()),
+                new Options(new Count()));
+        if (options == null)
+        {
+            printHelp(type);
+            System.out.println("Invalid " + type + " options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsCommandPreDefined(type, (Options) options);
+    }
+
+    static void printHelp(Command type)
+    {
+        printHelp(type.toString().toLowerCase());
+    }
+
+    static void printHelp(String type)
+    {
+        GroupedOptions.printOptions(System.out, type.toLowerCase(), new Uncertainty(), new Count());
+    }
+
+    static Runnable helpPrinter(final Command type)
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp(type);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefinedMixed.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefinedMixed.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefinedMixed.java
new file mode 100644
index 0000000..e5d4f80
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefinedMixed.java
@@ -0,0 +1,151 @@
+package org.apache.cassandra.stress.settings;
+/*
+ * 
+ * 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.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.stress.generate.PartitionGenerator;
+import org.apache.cassandra.stress.operations.OpDistributionFactory;
+import org.apache.cassandra.stress.operations.SampledOpDistributionFactory;
+import org.apache.cassandra.stress.operations.predefined.PredefinedOperation;
+import org.apache.cassandra.stress.util.Timer;
+
+import org.apache.commons.math3.util.Pair;
+
+// Settings unique to the mixed command type
+public class SettingsCommandPreDefinedMixed extends SettingsCommandPreDefined
+{
+
+    // Ratios for selecting commands - index for each Command, NaN indicates the command is not requested
+    private final List<Pair<Command, Double>> ratios;
+    private final DistributionFactory clustering;
+
+    public SettingsCommandPreDefinedMixed(Options options)
+    {
+        super(Command.MIXED, options);
+
+        clustering = options.clustering.get();
+        ratios = options.probabilities.ratios();
+        if (ratios.size() == 0)
+            throw new IllegalArgumentException("Must specify at least one command with a non-zero ratio");
+    }
+
+    public OpDistributionFactory getFactory(final StressSettings settings)
+    {
+        return new SampledOpDistributionFactory<Command>(ratios, clustering)
+        {
+            protected Operation get(Timer timer, PartitionGenerator generator, Command key)
+            {
+                return PredefinedOperation.operation(key, timer, generator, settings, add);
+            }
+
+            protected PartitionGenerator newGenerator()
+            {
+                return SettingsCommandPreDefinedMixed.this.newGenerator(settings);
+            }
+        };
+    }
+
+    // Option Declarations
+
+    static class Options extends SettingsCommandPreDefined.Options
+    {
+        static List<OptionEnumProbabilities.Opt<Command>> probabilityOptions = new ArrayList<>();
+        static
+        {
+            for (Command command : Command.values())
+            {
+                if (command.category == null)
+                    continue;
+                String defaultValue;
+                switch (command)
+                {
+                    case MIXED:
+                        continue;
+                    case READ:
+                    case WRITE:
+                        defaultValue = "1";
+                        break;
+                    default:
+                        defaultValue = null;
+                }
+                probabilityOptions.add(new OptionEnumProbabilities.Opt<>(command, defaultValue));
+            }
+        }
+
+        protected Options(SettingsCommand.Options parent)
+        {
+            super(parent);
+        }
+        final OptionDistribution clustering = new OptionDistribution("clustering=", "GAUSSIAN(1..10)", "Distribution clustering runs of operations of the same kind");
+        final OptionEnumProbabilities probabilities = new OptionEnumProbabilities<>(probabilityOptions, "ratio", "Specify the ratios for operations to perform; e.g. (read=2,write=1) will perform 2 reads for each write");
+
+        @Override
+        public List<? extends Option> options()
+        {
+            final List<Option> options = new ArrayList<>();
+            options.add(clustering);
+            options.add(probabilities);
+            options.addAll(super.options());
+            return options;
+        }
+
+    }
+
+    // CLI utility methods
+
+    public static SettingsCommandPreDefinedMixed build(String[] params)
+    {
+        GroupedOptions options = GroupedOptions.select(params,
+                new Options(new SettingsCommand.Uncertainty()),
+                new Options(new SettingsCommand.Count()));
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid MIXED options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsCommandPreDefinedMixed((Options) options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "mixed",
+                                    new Options(new SettingsCommand.Uncertainty()),
+                                    new Options(new SettingsCommand.Count()));
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandUser.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandUser.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandUser.java
new file mode 100644
index 0000000..f36296e
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandUser.java
@@ -0,0 +1,135 @@
+package org.apache.cassandra.stress.settings;
+/*
+ * 
+ * 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.File;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.math3.util.Pair;
+
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.StressProfile;
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.stress.generate.PartitionGenerator;
+import org.apache.cassandra.stress.operations.OpDistributionFactory;
+import org.apache.cassandra.stress.operations.SampledOpDistributionFactory;
+import org.apache.cassandra.stress.util.Timer;
+
+// Settings unique to the mixed command type
+public class SettingsCommandUser extends SettingsCommand
+{
+
+    // Ratios for selecting commands - index for each Command, NaN indicates the command is not requested
+    private final List<Pair<String, Double>> ratios;
+    private final DistributionFactory clustering;
+    public final StressProfile profile;
+
+    public SettingsCommandUser(Options options)
+    {
+        super(Command.USER, options.parent);
+
+        clustering = options.clustering.get();
+        ratios = options.ops.ratios();
+        profile = StressProfile.load(new File(options.profile.value()));
+
+        if (ratios.size() == 0)
+            throw new IllegalArgumentException("Must specify at least one command with a non-zero ratio");
+    }
+
+    public OpDistributionFactory getFactory(final StressSettings settings)
+    {
+        return new SampledOpDistributionFactory<String>(ratios, clustering)
+        {
+            protected Operation get(Timer timer, PartitionGenerator generator, String key)
+            {
+                if (key.equalsIgnoreCase("insert"))
+                    return profile.getInsert(timer, generator, settings);
+                return profile.getQuery(key, timer, generator, settings);
+            }
+
+            protected PartitionGenerator newGenerator()
+            {
+                return profile.newGenerator(settings);
+            }
+        };
+    }
+
+    static final class Options extends GroupedOptions
+    {
+        final SettingsCommand.Options parent;
+        protected Options(SettingsCommand.Options parent)
+        {
+            this.parent = parent;
+        }
+        final OptionDistribution clustering = new OptionDistribution("clustering=", "GAUSSIAN(1..10)", "Distribution clustering runs of operations of the same kind");
+        final OptionSimple profile = new OptionSimple("profile=", ".*", null, "Specify the path to a yaml cql3 profile", false);
+        final OptionAnyProbabilities ops = new OptionAnyProbabilities("ops", "Specify the ratios for inserts/queries to perform; e.g. ops(insert=2,<query1>=1) will perform 2 inserts for each query1");
+
+        @Override
+        public List<? extends Option> options()
+        {
+            final List<Option> options = new ArrayList<>();
+            options.add(clustering);
+            options.add(ops);
+            options.add(profile);
+            options.addAll(parent.options());
+            return options;
+        }
+
+    }
+
+    // CLI utility methods
+
+    public static SettingsCommandUser build(String[] params)
+    {
+        GroupedOptions options = GroupedOptions.select(params,
+                new Options(new Uncertainty()),
+                new Options(new Count()));
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid USER options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsCommandUser((Options) options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "user",
+                                    new Options(new Uncertainty()),
+                                    new Options(new Count()));
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsKey.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsKey.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsKey.java
index ab489be..017b106 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsKey.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsKey.java
@@ -26,23 +26,25 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.stress.generatedata.DataGenHexFromDistribution;
-import org.apache.cassandra.stress.generatedata.DataGenHexFromOpIndex;
-import org.apache.cassandra.stress.generatedata.DistributionFactory;
-import org.apache.cassandra.stress.generatedata.KeyGen;
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.stress.generate.SeedGenerator;
+import org.apache.cassandra.stress.generate.SeedRandomGenerator;
+import org.apache.cassandra.stress.generate.SeedSeriesGenerator;
 
 // Settings for key generation
 public class SettingsKey implements Serializable
 {
 
-    private final int keySize;
+    final int keySize;
     private final DistributionFactory distribution;
+    private final DistributionFactory clustering;
     private final long[] range;
 
     public SettingsKey(DistributionOptions options)
     {
         this.keySize = Integer.parseInt(options.size.value());
         this.distribution = options.dist.get();
+        this.clustering = options.clustering.get();
         this.range = null;
     }
 
@@ -50,8 +52,9 @@ public class SettingsKey implements Serializable
     {
         this.keySize = Integer.parseInt(options.size.value());
         this.distribution = null;
+        this.clustering = null;
         String[] bounds = options.populate.value().split("\\.\\.+");
-        this.range = new long[] { Long.parseLong(bounds[0]), Long.parseLong(bounds[1]) };
+        this.range = new long[] { OptionDistribution.parseLong(bounds[0]), OptionDistribution.parseLong(bounds[1]) };
     }
 
     // Option Declarations
@@ -59,6 +62,7 @@ public class SettingsKey implements Serializable
     private static final class DistributionOptions extends GroupedOptions
     {
         final OptionDistribution dist;
+        final OptionDistribution clustering = new OptionDistribution("cluster=", "fixed(1)", "Keys are clustered in adjacent value runs of this size");
         final OptionSimple size = new OptionSimple("size=", "[0-9]+", "10", "Key size in bytes", false);
 
         public DistributionOptions(String defaultLimit)
@@ -69,7 +73,7 @@ public class SettingsKey implements Serializable
         @Override
         public List<? extends Option> options()
         {
-            return Arrays.asList(dist, size);
+            return Arrays.asList(dist, size, clustering);
         }
     }
 
@@ -80,7 +84,7 @@ public class SettingsKey implements Serializable
 
         public PopulateOptions(String defaultLimit)
         {
-            populate = new OptionSimple("populate=", "[0-9]+\\.\\.+[0-9]+",
+            populate = new OptionSimple("populate=", "[0-9]+\\.\\.+[0-9]+[MBK]?",
                     "1.." + defaultLimit,
                     "Populate all keys in sequence", true);
         }
@@ -92,12 +96,9 @@ public class SettingsKey implements Serializable
         }
     }
 
-    public KeyGen newKeyGen()
+    public SeedGenerator newSeedGenerator()
     {
-        return new KeyGen(range == null
-                            ? new DataGenHexFromDistribution(distribution.get())
-                            : new DataGenHexFromOpIndex(range[0], range[1]),
-                          keySize);
+        return range == null ? new SeedRandomGenerator(distribution.get(), clustering.get()) : new SeedSeriesGenerator(range[0], range[1]);
     }
 
     // CLI Utility Methods

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java
index 8a5975f..5735f9d 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java
@@ -27,7 +27,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.stress.generatedata.Distribution;
+import org.apache.cassandra.stress.generate.Distribution;
 
 public class SettingsMisc implements Serializable
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
index 95c3da3..1aa745c 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
@@ -161,8 +161,10 @@ public class SettingsMode implements Serializable
         String[] params = clArgs.remove("-mode");
         if (params == null)
         {
-            ThriftOptions opts = new ThriftOptions();
-            opts.smart.accept("smart");
+            Cql3NativeOptions opts = new Cql3NativeOptions();
+            opts.accept("cql3");
+            opts.accept("native");
+            opts.accept("prepared");
             return new SettingsMode(opts);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
index ffafb67..5fb2bb2 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
@@ -25,40 +25,51 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.thrift.ConsistencyLevel;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class SettingsSchema implements Serializable
 {
 
-    public static final String DEFAULT_COMPARATOR = "AsciiType";
     public static final String DEFAULT_VALIDATOR  = "BytesType";
 
     private final String replicationStrategy;
     private final Map<String, String> replicationStrategyOptions;
 
-    private final IndexType indexType;
     private final String compression;
     private final String compactionStrategy;
     private final Map<String, String> compactionStrategyOptions;
     public final String keyspace;
 
-    public SettingsSchema(Options options)
+    public SettingsSchema(Options options, SettingsCommand command)
     {
+        if (command instanceof SettingsCommandUser)
+        {
+            if (options.compaction.setByUser() || options.keyspace.setByUser() || options.compression.setByUser() || options.replication.setByUser())
+                throw new IllegalArgumentException("Cannot provide command line schema settings if a user profile is provided");
+
+            keyspace = ((SettingsCommandUser) command).profile.keyspaceName;
+        }
+        else
+            keyspace = options.keyspace.value();
+
         replicationStrategy = options.replication.getStrategy();
         replicationStrategyOptions = options.replication.getOptions();
-        if (options.index.setByUser())
-            indexType = IndexType.valueOf(options.index.value().toUpperCase());
-        else
-            indexType = null;
         compression = options.compression.value();
         compactionStrategy = options.compaction.getStrategy();
         compactionStrategyOptions = options.compaction.getOptions();
-        keyspace = options.keyspace.value();
     }
 
     public void createKeySpaces(StressSettings settings)
     {
-        createKeySpacesThrift(settings);
+        if (!(settings.command instanceof SettingsCommandUser))
+        {
+            createKeySpacesThrift(settings);
+        }
+        else
+        {
+            ((SettingsCommandUser) settings.command).profile.maybeCreateSchema(settings);
+        }
     }
 
 
@@ -71,7 +82,7 @@ public class SettingsSchema implements Serializable
 
         // column family for standard columns
         CfDef standardCfDef = new CfDef(keyspace, "Standard1");
-        Map<String, String> compressionOptions = new HashMap<String, String>();
+        Map<String, String> compressionOptions = new HashMap<>();
         if (compression != null)
             compressionOptions.put("sstable_compression", compression);
 
@@ -80,28 +91,8 @@ public class SettingsSchema implements Serializable
                 .setDefault_validation_class(DEFAULT_VALIDATOR)
                 .setCompression_options(compressionOptions);
 
-        if (!settings.columns.useTimeUUIDComparator)
-        {
-            for (int i = 0; i < settings.columns.maxColumnsPerKey; i++)
-            {
-                standardCfDef.addToColumn_metadata(new ColumnDef(ByteBufferUtil.bytes("C" + i), "BytesType"));
-            }
-        }
-
-        if (indexType != null)
-        {
-            ColumnDef standardColumn = new ColumnDef(ByteBufferUtil.bytes("C1"), "BytesType");
-            standardColumn.setIndex_type(indexType).setIndex_name("Idx1");
-            standardCfDef.setColumn_metadata(Arrays.asList(standardColumn));
-        }
-
-        // column family with super columns
-        CfDef superCfDef = new CfDef(keyspace, "Super1")
-                .setColumn_type("Super");
-        superCfDef.setComparator_type(DEFAULT_COMPARATOR)
-                .setSubcomparator_type(comparator)
-                .setDefault_validation_class(DEFAULT_VALIDATOR)
-                .setCompression_options(compressionOptions);
+        for (int i = 0; i < settings.columns.names.size(); i++)
+            standardCfDef.addToColumn_metadata(new ColumnDef(settings.columns.names.get(i), "BytesType"));
 
         // column family for standard counters
         CfDef counterCfDef = new CfDef(keyspace, "Counter1")
@@ -109,13 +100,6 @@ public class SettingsSchema implements Serializable
                 .setDefault_validation_class("CounterColumnType")
                 .setCompression_options(compressionOptions);
 
-        // column family with counter super columns
-        CfDef counterSuperCfDef = new CfDef(keyspace, "SuperCounter1")
-                .setComparator_type(comparator)
-                .setDefault_validation_class("CounterColumnType")
-                .setColumn_type("Super")
-                .setCompression_options(compressionOptions);
-
         ksdef.setName(keyspace);
         ksdef.setStrategy_class(replicationStrategy);
 
@@ -127,19 +111,15 @@ public class SettingsSchema implements Serializable
         if (compactionStrategy != null)
         {
             standardCfDef.setCompaction_strategy(compactionStrategy);
-            superCfDef.setCompaction_strategy(compactionStrategy);
             counterCfDef.setCompaction_strategy(compactionStrategy);
-            counterSuperCfDef.setCompaction_strategy(compactionStrategy);
             if (!compactionStrategyOptions.isEmpty())
             {
                 standardCfDef.setCompaction_strategy_options(compactionStrategyOptions);
-                superCfDef.setCompaction_strategy_options(compactionStrategyOptions);
                 counterCfDef.setCompaction_strategy_options(compactionStrategyOptions);
-                counterSuperCfDef.setCompaction_strategy_options(compactionStrategyOptions);
             }
         }
 
-        ksdef.setCf_defs(new ArrayList<>(Arrays.asList(standardCfDef, superCfDef, counterCfDef, counterSuperCfDef)));
+        ksdef.setCf_defs(new ArrayList<>(Arrays.asList(standardCfDef, counterCfDef)));
 
         Cassandra.Client client = settings.getRawThriftClient(false);
 
@@ -191,24 +171,23 @@ public class SettingsSchema implements Serializable
     {
         final OptionReplication replication = new OptionReplication();
         final OptionCompaction compaction = new OptionCompaction();
-        final OptionSimple index = new OptionSimple("index=", "KEYS|CUSTOM|COMPOSITES", null, "Type of index to create on needed column families (KEYS)", false);
         final OptionSimple keyspace = new OptionSimple("keyspace=", ".*", "Keyspace1", "The keyspace name to use", false);
         final OptionSimple compression = new OptionSimple("compression=", ".*", null, "Specify the compression to use for sstable, default:no compression", false);
 
         @Override
         public List<? extends Option> options()
         {
-            return Arrays.asList(replication, index, keyspace, compaction, compression);
+            return Arrays.asList(replication, keyspace, compaction, compression);
         }
     }
 
     // CLI Utility Methods
 
-    public static SettingsSchema get(Map<String, String[]> clArgs)
+    public static SettingsSchema get(Map<String, String[]> clArgs, SettingsCommand command)
     {
         String[] params = clArgs.remove("-schema");
         if (params == null)
-            return new SettingsSchema(new Options());
+            return new SettingsSchema(new Options(), command);
 
         GroupedOptions options = GroupedOptions.select(params, new Options());
         if (options == null)
@@ -217,7 +196,7 @@ public class SettingsSchema implements Serializable
             System.out.println("Invalid -schema options provided, see output for valid options");
             System.exit(1);
         }
-        return new SettingsSchema((Options) options);
+        return new SettingsSchema((Options) options, command);
     }
 
     public static void printHelp()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
index 396095d..8b0ef6a 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
@@ -111,7 +111,7 @@ public class SettingsTransport implements Serializable
 
     // Option Declarations
 
-    static class TOptions extends GroupedOptions
+    static class TOptions extends GroupedOptions implements Serializable
     {
         final OptionSimple factory = new OptionSimple("factory=", ".*", TFramedTransportFactory.class.getName(), "Fully-qualified ITransportFactory class name for creating a connection. Note: For Thrift over SSL, use org.apache.cassandra.thrift.SSLTransportFactory.", false);
         final OptionSimple trustStore = new OptionSimple("truststore=", ".*", null, "SSL: full path to truststore", false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java b/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
index a1d68b4..ab57289 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
@@ -29,6 +29,7 @@ import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.stress.util.JavaDriverClient;
 import org.apache.cassandra.stress.util.SimpleThriftClient;
 import org.apache.cassandra.stress.util.SmartThriftClient;
+import org.apache.cassandra.stress.util.ThriftClient;
 import org.apache.cassandra.thrift.Cassandra;
 import org.apache.cassandra.thrift.InvalidRequestException;
 import org.apache.cassandra.transport.SimpleClient;
@@ -37,7 +38,6 @@ import org.apache.thrift.transport.TTransport;
 
 public class StressSettings implements Serializable
 {
-
     public final SettingsCommand command;
     public final SettingsRate rate;
     public final SettingsKey keys;
@@ -65,7 +65,24 @@ public class StressSettings implements Serializable
         this.sendToDaemon = sendToDaemon;
     }
 
-    public SmartThriftClient getSmartThriftClient()
+    private SmartThriftClient tclient;
+
+    /**
+     * Thrift client connection
+     * @return cassandra client connection
+     */
+    public synchronized ThriftClient getThriftClient()
+    {
+        if (mode.api != ConnectionAPI.THRIFT_SMART)
+            return getSimpleThriftClient();
+
+        if (tclient == null)
+            tclient = getSmartThriftClient();
+
+        return tclient;
+    }
+
+    private SmartThriftClient getSmartThriftClient()
     {
         Metadata metadata = getJavaDriverClient().getCluster().getMetadata();
         return new SmartThriftClient(this, schema.keyspace, metadata);
@@ -75,7 +92,7 @@ public class StressSettings implements Serializable
      * Thrift client connection
      * @return cassandra client connection
      */
-    public SimpleThriftClient getThriftClient()
+    private SimpleThriftClient getSimpleThriftClient()
     {
         return new SimpleThriftClient(getRawThriftClient(node.randomNode(), true));
     }
@@ -139,6 +156,11 @@ public class StressSettings implements Serializable
 
     public JavaDriverClient getJavaDriverClient()
     {
+        return getJavaDriverClient(true);
+    }
+
+    public JavaDriverClient getJavaDriverClient(boolean setKeyspace)
+    {
         if (client != null)
             return client;
 
@@ -153,7 +175,9 @@ public class StressSettings implements Serializable
                 EncryptionOptions.ClientEncryptionOptions encOptions = transport.getEncryptionOptions();
                 JavaDriverClient c = new JavaDriverClient(currentNode, port.nativePort, encOptions);
                 c.connect(mode.compression());
-                c.execute("USE \"" + schema.keyspace + "\";", org.apache.cassandra.db.ConsistencyLevel.ONE);
+                if (setKeyspace)
+                    c.execute("USE \"" + schema.keyspace + "\";", org.apache.cassandra.db.ConsistencyLevel.ONE);
+
                 return client = c;
             }
         }
@@ -165,7 +189,7 @@ public class StressSettings implements Serializable
 
     public void maybeCreateKeyspaces()
     {
-        if (command.type == Command.WRITE || command.type == Command.COUNTER_WRITE)
+        if (command.type == Command.WRITE || command.type == Command.COUNTER_WRITE || command.type == Command.USER)
             schema.createKeySpaces(this);
 
     }
@@ -202,7 +226,7 @@ public class StressSettings implements Serializable
         SettingsLog log = SettingsLog.get(clArgs);
         SettingsMode mode = SettingsMode.get(clArgs);
         SettingsNode node = SettingsNode.get(clArgs);
-        SettingsSchema schema = SettingsSchema.get(clArgs);
+        SettingsSchema schema = SettingsSchema.get(clArgs, command);
         SettingsTransport transport = SettingsTransport.get(clArgs);
         if (!clArgs.isEmpty())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/settings/ValidationType.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/ValidationType.java b/tools/stress/src/org/apache/cassandra/stress/settings/ValidationType.java
new file mode 100644
index 0000000..c22242d
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/ValidationType.java
@@ -0,0 +1,8 @@
+package org.apache.cassandra.stress.settings;
+
+public enum ValidationType
+{
+
+    NOT_FAIL, NON_ZERO, SUBSET, EQUAL
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/75364296/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
index f216c05..c901461 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
@@ -18,6 +18,8 @@
 package org.apache.cassandra.stress.util;
 
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import javax.net.ssl.SSLContext;
 
 import com.datastax.driver.core.*;
@@ -40,6 +42,8 @@ public class JavaDriverClient
     private Cluster cluster;
     private Session session;
 
+    private static final ConcurrentMap<String, PreparedStatement> stmts = new ConcurrentHashMap<>();
+
     public JavaDriverClient(String host, int port)
     {
         this(host, port, new EncryptionOptions.ClientEncryptionOptions());
@@ -54,7 +58,18 @@ public class JavaDriverClient
 
     public PreparedStatement prepare(String query)
     {
-        return getSession().prepare(query);
+        PreparedStatement stmt = stmts.get(query);
+        if (stmt != null)
+            return stmt;
+        synchronized (stmts)
+        {
+            stmt = stmts.get(query);
+            if (stmt != null)
+                return stmt;
+            stmt = getSession().prepare(query);
+            stmts.put(query, stmt);
+        }
+        return stmt;
     }
 
     public void connect(ProtocolOptions.Compression compression) throws Exception
@@ -116,7 +131,7 @@ public class JavaDriverClient
      * @param cl
      * @return
      */
-    ConsistencyLevel from(org.apache.cassandra.db.ConsistencyLevel cl)
+    public static ConsistencyLevel from(org.apache.cassandra.db.ConsistencyLevel cl)
     {
         switch (cl)
         {