You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by xe...@apache.org on 2013/12/24 03:08:40 UTC

[1/6] Improve Stress Tool patch by Benedict; reviewed by Pavel Yaskevich for CASSANDRA-6199

Updated Branches:
  refs/heads/trunk 34235ad7b -> 2e1e98ad0


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/SmartThriftClient.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/SmartThriftClient.java b/tools/stress/src/org/apache/cassandra/stress/util/SmartThriftClient.java
new file mode 100644
index 0000000..99fa452
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/SmartThriftClient.java
@@ -0,0 +1,235 @@
+package org.apache.cassandra.stress.util;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.datastax.driver.core.Host;
+import com.datastax.driver.core.Metadata;
+import org.apache.cassandra.stress.settings.StressSettings;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
+
+public class SmartThriftClient implements ThriftClient
+{
+
+    final String keyspace;
+    final Metadata metadata;
+    final StressSettings settings;
+    final ConcurrentHashMap<Host, ConcurrentLinkedQueue<Client>> cache = new ConcurrentHashMap<>();
+
+    final AtomicInteger queryIdCounter = new AtomicInteger();
+    final ConcurrentHashMap<Integer, String> queryStrings = new ConcurrentHashMap<>();
+    final ConcurrentHashMap<String, Integer> queryIds = new ConcurrentHashMap<>();
+
+    public SmartThriftClient(StressSettings settings, String keyspace, Metadata metadata)
+    {
+        this.metadata = metadata;
+        this.keyspace = keyspace;
+        this.settings = settings;
+    }
+
+    private final AtomicInteger roundrobin = new AtomicInteger();
+
+    private Integer getId(String query)
+    {
+        Integer r;
+        if ((r = queryIds.get(query)) != null)
+            return r;
+        r = queryIdCounter.incrementAndGet();
+        if (queryIds.putIfAbsent(query, r) == null)
+            return r;
+        queryStrings.put(r, query);
+        return queryIds.get(query);
+    }
+
+    final class Client
+    {
+        final Cassandra.Client client;
+        final Host host;
+        final Map<Integer, Integer> queryMap = new HashMap<>();
+
+        Client(Cassandra.Client client, Host host)
+        {
+            this.client = client;
+            this.host = host;
+        }
+
+        Integer get(Integer id, boolean cql3) throws TException
+        {
+            Integer serverId = queryMap.get(id);
+            if (serverId != null)
+                return serverId;
+            prepare(id, cql3);
+            return queryMap.get(id);
+        }
+
+       void prepare(Integer id, boolean cql3) throws TException
+       {
+           String query;
+           while ( null == (query = queryStrings.get(id)) ) ;
+           if (cql3)
+           {
+               Integer serverId = client.prepare_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE).itemId;
+               queryMap.put(id, serverId);
+           }
+           else
+           {
+               Integer serverId = client.prepare_cql_query(ByteBufferUtil.bytes(query), Compression.NONE).itemId;
+               queryMap.put(id, serverId);
+           }
+       }
+    }
+
+    private Client get(ByteBuffer pk)
+    {
+        Set<Host> hosts = metadata.getReplicas(keyspace, pk);
+        int count = roundrobin.incrementAndGet() % hosts.size();
+        if (count < 0)
+            count = -count;
+        Iterator<Host> iter = hosts.iterator();
+        while (count > 0 && iter.hasNext())
+            iter.next();
+        Host host = iter.next();
+        ConcurrentLinkedQueue<Client> q = cache.get(host);
+        if (q == null)
+        {
+            ConcurrentLinkedQueue<Client> newQ = new ConcurrentLinkedQueue<Client>();
+            q = cache.putIfAbsent(host, newQ);
+            if (q == null)
+                q = newQ;
+        }
+        Client tclient = q.poll();
+        if (tclient != null)
+            return tclient;
+        return new Client(settings.getRawThriftClient(host.getAddress().getHostAddress()), host);
+    }
+
+    @Override
+    public void batch_mutate(Map<ByteBuffer, Map<String, List<Mutation>>> record, ConsistencyLevel consistencyLevel) throws TException
+    {
+        for (Map.Entry<ByteBuffer, Map<String, List<Mutation>>> e : record.entrySet())
+        {
+            Client client = get(e.getKey());
+            try
+            {
+                client.client.batch_mutate(Collections.singletonMap(e.getKey(), e.getValue()), consistencyLevel);
+            } finally
+            {
+                cache.get(client.host).add(client);
+            }
+        }
+    }
+
+    @Override
+    public List<ColumnOrSuperColumn> get_slice(ByteBuffer key, ColumnParent parent, SlicePredicate predicate, ConsistencyLevel consistencyLevel) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        Client client = get(key);
+        try
+        {
+            return client.client.get_slice(key, parent, predicate, consistencyLevel);
+        } finally
+        {
+            cache.get(client.host).add(client);
+        }
+    }
+
+    @Override
+    public void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        Client client = get(key);
+        try
+        {
+            client.client.insert(key, column_parent, column, consistency_level);
+        } finally
+        {
+            cache.get(client.host).add(client);
+        }
+    }
+
+    @Override
+    public CqlResult execute_cql_query(String query, ByteBuffer key, Compression compression) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    {
+        Client client = get(key);
+        try
+        {
+            return client.client.execute_cql_query(ByteBufferUtil.bytes(query), compression);
+        } finally
+        {
+            cache.get(client.host).add(client);
+        }
+    }
+
+    @Override
+    public CqlResult execute_cql3_query(String query, ByteBuffer key, Compression compression, ConsistencyLevel consistency) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    {
+        Client client = get(key);
+        try
+        {
+            return client.client.execute_cql3_query(ByteBufferUtil.bytes(query), compression, consistency);
+        } finally
+        {
+            cache.get(client.host).add(client);
+        }
+    }
+
+    @Override
+    public Integer prepare_cql3_query(String query, Compression compression) throws InvalidRequestException, TException
+    {
+        return getId(query);
+    }
+
+    @Override
+    public CqlResult execute_prepared_cql3_query(int queryId, ByteBuffer key, List<ByteBuffer> values, ConsistencyLevel consistency) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    {
+        Client client = get(key);
+        try
+        {
+            return client.client.execute_prepared_cql3_query(client.get(queryId, true), values, consistency);
+        } finally
+        {
+            cache.get(client.host).add(client);
+        }
+    }
+
+    @Override
+    public Integer prepare_cql_query(String query, Compression compression) throws InvalidRequestException, TException
+    {
+        return getId(query);
+    }
+
+    @Override
+    public CqlResult execute_prepared_cql_query(int queryId, ByteBuffer key, List<ByteBuffer> values) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    {
+        Client client = get(key);
+        try
+        {
+            return client.client.execute_prepared_cql_query(client.get(queryId, true), values);
+        } finally
+        {
+            cache.get(client.host).add(client);
+        }
+    }
+
+    @Override
+    public Map<ByteBuffer, List<ColumnOrSuperColumn>> multiget_slice(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<KeySlice> get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<KeySlice> get_indexed_slices(ColumnParent column_parent, IndexClause index_clause, SlicePredicate column_predicate, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/ThriftClient.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/ThriftClient.java b/tools/stress/src/org/apache/cassandra/stress/util/ThriftClient.java
new file mode 100644
index 0000000..1ceca29
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/ThriftClient.java
@@ -0,0 +1,36 @@
+package org.apache.cassandra.stress.util;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.thrift.TException;
+
+public interface ThriftClient
+{
+
+    public void batch_mutate(Map<ByteBuffer, Map<String, List<Mutation>>> record, ConsistencyLevel consistencyLevel) throws TException;
+
+    List<ColumnOrSuperColumn> get_slice(ByteBuffer key, ColumnParent parent, SlicePredicate predicate, ConsistencyLevel consistencyLevel) throws InvalidRequestException, UnavailableException, TimedOutException, TException;
+
+    void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException;
+
+    Map<ByteBuffer, List<ColumnOrSuperColumn>> multiget_slice(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException;
+
+    List<KeySlice> get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException;
+
+    List<KeySlice> get_indexed_slices(ColumnParent column_parent, IndexClause index_clause, SlicePredicate column_predicate, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException;
+
+    Integer prepare_cql3_query(String query, Compression compression) throws InvalidRequestException, TException;
+
+    CqlResult execute_prepared_cql3_query(int itemId, ByteBuffer key, List<ByteBuffer> values, ConsistencyLevel consistency) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException;
+
+    CqlResult execute_cql_query(String query, ByteBuffer key, Compression compression) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException;
+
+    CqlResult execute_cql3_query(String query, ByteBuffer key, Compression compression, ConsistencyLevel consistency) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException;
+
+    Integer prepare_cql_query(String query, Compression compression) throws InvalidRequestException, TException;
+
+    CqlResult execute_prepared_cql_query(int itemId, ByteBuffer key, List<ByteBuffer> values) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException;
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/Timer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/Timer.java b/tools/stress/src/org/apache/cassandra/stress/util/Timer.java
new file mode 100644
index 0000000..c216561
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/Timer.java
@@ -0,0 +1,129 @@
+package org.apache.cassandra.stress.util;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+
+// a timer - this timer must be used by a single thread, and co-ordinates with other timers by
+public final class Timer
+{
+
+    private static final int SAMPLE_SIZE_SHIFT = 10;
+    private static final int SAMPLE_SIZE_MASK = (1 << SAMPLE_SIZE_SHIFT) - 1;
+
+    private final Random rnd = new Random();
+
+    // in progress snap start
+    private long sampleStartNanos;
+
+    // each entry is present with probability 1/p(opCount) or 1/(p(opCount)-1)
+    private final long[] sample = new long[1 << SAMPLE_SIZE_SHIFT];
+    private int opCount;
+
+    // aggregate info
+    private int keyCount;
+    private long total;
+    private long max;
+    private long maxStart;
+    private long upToDateAsOf;
+    private long lastSnap = System.nanoTime();
+
+    // communication with summary/logging thread
+    private volatile CountDownLatch reportRequest;
+    volatile TimingInterval report;
+    private volatile TimingInterval finalReport;
+
+    public void start(){
+        // decide if we're logging this event
+        sampleStartNanos = System.nanoTime();
+    }
+
+    private static int p(int index)
+    {
+        return 1 + (index >>> SAMPLE_SIZE_SHIFT);
+    }
+
+    public void stop(int keys)
+    {
+        maybeReport();
+        long now = System.nanoTime();
+        long time = now - sampleStartNanos;
+        if (rnd.nextInt(p(opCount)) == 0)
+            sample[index(opCount)] = time;
+        if (time > max)
+        {
+            maxStart = sampleStartNanos;
+            max = time;
+        }
+        total += time;
+        opCount += 1;
+        keyCount += keys;
+        upToDateAsOf = now;
+    }
+
+    private static int index(int count)
+    {
+        return count & SAMPLE_SIZE_MASK;
+    }
+
+    private TimingInterval buildReport()
+    {
+        final List<SampleOfLongs> sampleLatencies = Arrays.asList
+                (       new SampleOfLongs(Arrays.copyOf(sample, index(opCount)), p(opCount)),
+                        new SampleOfLongs(Arrays.copyOfRange(sample, index(opCount), Math.min(opCount, sample.length)), p(opCount) - 1)
+                );
+        final TimingInterval report = new TimingInterval(lastSnap, upToDateAsOf, max, maxStart, max, keyCount, total, opCount,
+                SampleOfLongs.merge(rnd, sampleLatencies, Integer.MAX_VALUE));
+        // reset counters
+        opCount = 0;
+        keyCount = 0;
+        total = 0;
+        max = 0;
+        lastSnap = upToDateAsOf;
+        return report;
+    }
+
+    // checks to see if a report has been requested, and if so produces the report, signals and clears the request
+    private void maybeReport()
+    {
+        if (reportRequest != null)
+        {
+            synchronized (this)
+            {
+                report = buildReport();
+                reportRequest.countDown();
+                reportRequest = null;
+            }
+        }
+    }
+
+    // checks to see if the timer is dead; if not requests a report, and otherwise fulfills the request itself
+    synchronized void requestReport(CountDownLatch signal)
+    {
+        if (finalReport != null)
+        {
+            report = finalReport;
+            finalReport = new TimingInterval(0);
+            signal.countDown();
+        }
+        else
+            reportRequest = signal;
+    }
+
+    // closes the timer; if a request is outstanding, it furnishes the request, otherwise it populates finalReport
+    public synchronized void close()
+    {
+        if (reportRequest == null)
+            finalReport = buildReport();
+        else
+        {
+            finalReport = new TimingInterval(0);
+            report = buildReport();
+            reportRequest.countDown();
+            reportRequest = null;
+        }
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/Timing.java b/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
new file mode 100644
index 0000000..6f5052f
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
@@ -0,0 +1,72 @@
+package org.apache.cassandra.stress.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+// relatively simple timing class for getting a uniform sample of latencies, and saving other metrics
+// ensures accuracy of timing by having single threaded timers that are check-pointed by the snapping thread,
+// which waits for them to report back. They report back the data up to the last event prior to the check-point.
+// if the threads are blocked/paused this may mean a period of time longer than the checkpoint elapses, but that all
+// metrics calculated over the interval are accurate
+public class Timing
+{
+
+    private final CopyOnWriteArrayList<Timer> timers = new CopyOnWriteArrayList<>();
+    private volatile TimingInterval history;
+    private final Random rnd = new Random();
+
+    // TIMING
+
+    private TimingInterval snapInterval(Random rnd) throws InterruptedException
+    {
+        final Timer[] timers = this.timers.toArray(new Timer[0]);
+        final CountDownLatch ready = new CountDownLatch(timers.length);
+        for (int i = 0 ; i < timers.length ; i++)
+        {
+            final Timer timer = timers[i];
+            timer.requestReport(ready);
+        }
+
+        // TODO fail gracefully after timeout if a thread is stuck
+        if (!ready.await(2L, TimeUnit.MINUTES))
+            throw new RuntimeException("Timed out waiting for a timer thread - seems one got stuck");
+
+        // reports have been filled in by timer threadCount, so merge
+        List<TimingInterval> intervals = new ArrayList<>();
+        for (Timer timer : timers)
+            intervals.add(timer.report);
+
+        return TimingInterval.merge(rnd, intervals, Integer.MAX_VALUE, history.endNanos());
+    }
+
+    // build a new timer and add it to the set of running timers
+    public Timer newTimer()
+    {
+        final Timer timer = new Timer();
+        timers.add(timer);
+        return timer;
+    }
+
+    public void start()
+    {
+        history = new TimingInterval(System.nanoTime());
+    }
+
+    public TimingInterval snapInterval() throws InterruptedException
+    {
+        final TimingInterval interval = snapInterval(rnd);
+        history = TimingInterval.merge(rnd, Arrays.asList(interval, history), 50000, history.startNanos());
+        return interval;
+    }
+
+    public TimingInterval getHistory()
+    {
+        return history;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java b/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java
new file mode 100644
index 0000000..04fb044
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java
@@ -0,0 +1,132 @@
+package org.apache.cassandra.stress.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+// represents measurements taken over an interval of time
+// used for both single timer results and merged timer results
+public final class TimingInterval
+{
+    // nanos
+    private final long start;
+    private final long end;
+    public final long maxLatency;
+    public final long pauseLength;
+    public final long pauseStart;
+    public final long totalLatency;
+
+    // discrete
+    public final long keyCount;
+    public final long operationCount;
+
+    final SampleOfLongs sample;
+
+    TimingInterval(long time)
+    {
+        start = end = time;
+        maxLatency = totalLatency = 0;
+        keyCount = operationCount = 0;
+        pauseStart = pauseLength = 0;
+        sample = new SampleOfLongs(new long[0], 1d);
+    }
+    TimingInterval(long start, long end, long maxLatency, long pauseStart, long pauseLength, long keyCount, long totalLatency, long operationCount, SampleOfLongs sample)
+    {
+        this.start = start;
+        this.end = Math.max(end, start);
+        this.maxLatency = maxLatency;
+        this.keyCount = keyCount;
+        this.totalLatency = totalLatency;
+        this.operationCount = operationCount;
+        this.pauseStart = pauseStart;
+        this.pauseLength = pauseLength;
+        this.sample = sample;
+    }
+
+    // merge multiple timer intervals together
+    static TimingInterval merge(Random rnd, List<TimingInterval> intervals, int maxSamples, long start)
+    {
+        int operationCount = 0, keyCount = 0;
+        long maxLatency = 0, totalLatency = 0;
+        List<SampleOfLongs> latencies = new ArrayList<>();
+        long end = 0;
+        long pauseStart = 0, pauseEnd = Long.MAX_VALUE;
+        for (TimingInterval interval : intervals)
+        {
+            end = Math.max(end, interval.end);
+            operationCount += interval.operationCount;
+            maxLatency = Math.max(interval.maxLatency, maxLatency);
+            totalLatency += interval.totalLatency;
+            keyCount += interval.keyCount;
+            latencies.addAll(Arrays.asList(interval.sample));
+            if (interval.pauseLength > 0)
+            {
+                pauseStart = Math.max(pauseStart, interval.pauseStart);
+                pauseEnd = Math.min(pauseEnd, interval.pauseStart + interval.pauseLength);
+            }
+        }
+        if (pauseEnd < pauseStart)
+            pauseEnd = pauseStart = 0;
+        return new TimingInterval(start, end, maxLatency, pauseStart, pauseEnd - pauseStart, keyCount, totalLatency, operationCount,
+                SampleOfLongs.merge(rnd, latencies, maxSamples));
+
+    }
+
+    public double realOpRate()
+    {
+        return operationCount / ((end - start) * 0.000000001d);
+    }
+
+    public double adjustedOpRate()
+    {
+        return operationCount / ((end - (start + pauseLength)) * 0.000000001d);
+    }
+
+    public double keyRate()
+    {
+        return keyCount / ((end - start) * 0.000000001d);
+    }
+
+    public double meanLatency()
+    {
+        return (totalLatency / (double) operationCount) * 0.000001d;
+    }
+
+    public double maxLatency()
+    {
+        return maxLatency * 0.000001d;
+    }
+
+    public long runTime()
+    {
+        return (end - start) / 1000000;
+    }
+
+    public double medianLatency()
+    {
+        return sample.medianLatency();
+    }
+
+    // 0 < rank < 1
+    public double rankLatency(float rank)
+    {
+        return sample.rankLatency(rank);
+    }
+
+    public final long endNanos()
+    {
+        return end;
+    }
+
+    public final long endMillis()
+    {
+        return end / 1000000;
+    }
+
+    public long startNanos()
+    {
+        return start;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/Uncertainty.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/Uncertainty.java b/tools/stress/src/org/apache/cassandra/stress/util/Uncertainty.java
new file mode 100644
index 0000000..ac2d803
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/Uncertainty.java
@@ -0,0 +1,81 @@
+package org.apache.cassandra.stress.util;
+
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+
+// TODO: do not assume normal distribution of measurements.
+public class Uncertainty
+{
+
+    private int measurements;
+    private double sumsquares;
+    private double sum;
+    private double stdev;
+    private double mean;
+    private double uncertainty;
+
+    private CopyOnWriteArrayList<WaitForTargetUncertainty> waiting = new CopyOnWriteArrayList<>();
+
+    private static final class WaitForTargetUncertainty
+    {
+        final double targetUncertainty;
+        final int minMeasurements;
+        final int maxMeasurements;
+        final CountDownLatch latch = new CountDownLatch(1);
+
+        private WaitForTargetUncertainty(double targetUncertainty, int minMeasurements, int maxMeasurements)
+        {
+            this.targetUncertainty = targetUncertainty;
+            this.minMeasurements = minMeasurements;
+            this.maxMeasurements = maxMeasurements;
+        }
+
+        void await() throws InterruptedException
+        {
+            latch.await();
+        }
+
+    }
+
+    public void update(double value)
+    {
+        measurements++;
+        sumsquares += value * value;
+        sum += value;
+        mean = sum / measurements;
+        stdev = Math.sqrt((sumsquares / measurements) - (mean * mean));
+        uncertainty = (stdev / Math.sqrt(measurements)) / mean;
+
+        for (WaitForTargetUncertainty waiter : waiting)
+        {
+            if ((uncertainty < waiter.targetUncertainty && measurements >= waiter.minMeasurements) || (measurements >= waiter.maxMeasurements))
+            {
+                waiter.latch.countDown();
+                // can safely remove as working over snapshot with COWArrayList
+                waiting.remove(waiter);
+            }
+        }
+    }
+
+    public void await(double targetUncertainty, int minMeasurements, int maxMeasurements) throws InterruptedException
+    {
+        final WaitForTargetUncertainty wait = new WaitForTargetUncertainty(targetUncertainty, minMeasurements, maxMeasurements);
+        waiting.add(wait);
+        wait.await();
+    }
+
+    public double getUncertainty()
+    {
+        return uncertainty;
+    }
+
+    public void wakeAll()
+    {
+        for (WaitForTargetUncertainty waiting : this.waiting)
+        {
+            waiting.latch.countDown();
+            this.waiting.remove(waiting);
+        }
+    }
+
+}


[6/6] git commit: Improve Stress Tool patch by Benedict; reviewed by Pavel Yaskevich for CASSANDRA-6199

Posted by xe...@apache.org.
Improve Stress Tool
patch by Benedict; reviewed by Pavel Yaskevich for CASSANDRA-6199


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2e1e98ad
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2e1e98ad
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2e1e98ad

Branch: refs/heads/trunk
Commit: 2e1e98ad04c81900524763eddf560edc55dfb299
Parents: 34235ad
Author: belliottsmith <gi...@sub.laerad.com>
Authored: Tue Dec 24 00:33:38 2013 +0000
Committer: Pavel Yaskevich <xe...@apache.org>
Committed: Mon Dec 23 18:05:43 2013 -0800

----------------------------------------------------------------------
 build.xml                                       |   6 +
 lib/commons-math3-3.2.jar                       | Bin 0 -> 1692782 bytes
 tools/bin/cassandra-stressd                     |  34 +-
 ...2.0.0-rc2-SNAPSHOT-jar-with-dependencies.jar | Bin 0 -> 5869229 bytes
 ...cassandra-driver-core-2.0.0-rc2-SNAPSHOT.jar | Bin 0 -> 490145 bytes
 .../org/apache/cassandra/stress/Operation.java  | 204 +++++
 .../org/apache/cassandra/stress/Session.java    | 841 -------------------
 .../src/org/apache/cassandra/stress/Stress.java |  63 +-
 .../apache/cassandra/stress/StressAction.java   | 639 +++++++++-----
 .../apache/cassandra/stress/StressMetrics.java  | 178 ++++
 .../apache/cassandra/stress/StressServer.java   |  90 +-
 .../cassandra/stress/StressStatistics.java      | 126 ---
 .../cassandra/stress/generatedata/DataGen.java  |  18 +
 .../stress/generatedata/DataGenBytesRandom.java |  24 +
 .../stress/generatedata/DataGenFactory.java     |   9 +
 .../stress/generatedata/DataGenHex.java         |  39 +
 .../DataGenHexFromDistribution.java             |  45 +
 .../generatedata/DataGenHexFromOpIndex.java     |  27 +
 .../generatedata/DataGenStringDictionary.java   |  84 ++
 .../generatedata/DataGenStringRepeats.java      |  69 ++
 .../stress/generatedata/Distribution.java       |  19 +
 .../generatedata/DistributionBoundApache.java   |  42 +
 .../generatedata/DistributionFactory.java       |  10 +
 .../stress/generatedata/DistributionFixed.java  |  25 +
 .../generatedata/DistributionOffsetApache.java  |  40 +
 .../generatedata/DistributionSeqBatch.java      |  47 ++
 .../cassandra/stress/generatedata/KeyGen.java   |  33 +
 .../cassandra/stress/generatedata/RowGen.java   |  31 +
 .../generatedata/RowGenDistributedSize.java     |  84 ++
 .../stress/operations/CQLOperation.java         |  96 ---
 .../stress/operations/CounterAdder.java         | 141 ----
 .../stress/operations/CounterGetter.java        | 152 ----
 .../stress/operations/CqlCounterAdder.java      |  98 +--
 .../stress/operations/CqlCounterGetter.java     |  98 +--
 .../operations/CqlIndexedRangeSlicer.java       | 224 ++---
 .../stress/operations/CqlInserter.java          | 124 +--
 .../stress/operations/CqlMultiGetter.java       |  19 +-
 .../stress/operations/CqlOperation.java         | 566 +++++++++++++
 .../stress/operations/CqlRangeSlicer.java       |  95 +--
 .../cassandra/stress/operations/CqlReader.java  | 121 +--
 .../stress/operations/IndexedRangeSlicer.java   | 135 ---
 .../cassandra/stress/operations/Inserter.java   | 135 ---
 .../stress/operations/MultiGetter.java          | 152 ----
 .../stress/operations/RangeSlicer.java          | 144 ----
 .../cassandra/stress/operations/Reader.java     | 159 ----
 .../stress/operations/ThriftCounterAdder.java   |  95 +++
 .../stress/operations/ThriftCounterGetter.java  |  75 ++
 .../operations/ThriftIndexedRangeSlicer.java    | 115 +++
 .../stress/operations/ThriftInserter.java       | 117 +++
 .../stress/operations/ThriftMultiGetter.java    |  81 ++
 .../stress/operations/ThriftRangeSlicer.java    |  86 ++
 .../stress/operations/ThriftReader.java         |  76 ++
 .../cassandra/stress/server/StressThread.java   |  77 --
 .../cassandra/stress/settings/CliOption.java    |  58 ++
 .../cassandra/stress/settings/Command.java      | 101 +++
 .../stress/settings/CommandCategory.java        |   8 +
 .../stress/settings/ConnectionAPI.java          |   7 +
 .../stress/settings/ConnectionStyle.java        |   9 +
 .../cassandra/stress/settings/CqlVersion.java   |  48 ++
 .../stress/settings/GroupedOptions.java         | 104 +++
 .../cassandra/stress/settings/Legacy.java       | 369 ++++++++
 .../cassandra/stress/settings/Option.java       |  24 +
 .../stress/settings/OptionDataGen.java          | 177 ++++
 .../stress/settings/OptionDistribution.java     | 340 ++++++++
 .../cassandra/stress/settings/OptionMulti.java  | 107 +++
 .../stress/settings/OptionReplication.java      | 114 +++
 .../cassandra/stress/settings/OptionSimple.java | 131 +++
 .../stress/settings/SettingsColumn.java         | 176 ++++
 .../stress/settings/SettingsCommand.java        | 159 ++++
 .../stress/settings/SettingsCommandMixed.java   | 184 ++++
 .../stress/settings/SettingsCommandMulti.java   |  69 ++
 .../cassandra/stress/settings/SettingsKey.java  | 130 +++
 .../cassandra/stress/settings/SettingsLog.java  |  92 ++
 .../cassandra/stress/settings/SettingsMisc.java | 200 +++++
 .../cassandra/stress/settings/SettingsMode.java | 154 ++++
 .../cassandra/stress/settings/SettingsNode.java | 103 +++
 .../cassandra/stress/settings/SettingsPort.java |  70 ++
 .../cassandra/stress/settings/SettingsRate.java | 116 +++
 .../stress/settings/SettingsSchema.java         | 236 ++++++
 .../stress/settings/SettingsTransport.java      | 121 +++
 .../stress/settings/StressSettings.java         | 239 ++++++
 .../cassandra/stress/util/CassandraClient.java  |  34 -
 .../cassandra/stress/util/JavaDriverClient.java | 148 ++++
 .../apache/cassandra/stress/util/Operation.java | 334 --------
 .../cassandra/stress/util/SampleOfLongs.java    | 107 +++
 .../stress/util/SimpleThriftClient.java         |  90 ++
 .../stress/util/SmartThriftClient.java          | 235 ++++++
 .../cassandra/stress/util/ThriftClient.java     |  36 +
 .../org/apache/cassandra/stress/util/Timer.java | 129 +++
 .../apache/cassandra/stress/util/Timing.java    |  72 ++
 .../cassandra/stress/util/TimingInterval.java   | 132 +++
 .../cassandra/stress/util/Uncertainty.java      |  81 ++
 92 files changed, 7692 insertions(+), 3360 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 1ffb908..347323f 100644
--- a/build.xml
+++ b/build.xml
@@ -37,6 +37,7 @@
     <property name="build.src.resources" value="${basedir}/src/resources"/>
     <property name="build.src.gen-java" value="${basedir}/src/gen-java"/>
     <property name="build.lib" value="${basedir}/lib"/>
+    <property name="build.tools.lib" value="${basedir}/tools/lib"/>
     <property name="build.dir" value="${basedir}/build"/>
     <property name="build.dir.lib" value="${basedir}/build/lib"/>
     <property name="build.test.dir" value="${build.dir}/test"/>
@@ -343,6 +344,7 @@
           <dependency groupId="commons-cli" artifactId="commons-cli" version="1.1"/>
           <dependency groupId="commons-codec" artifactId="commons-codec" version="1.2"/>
           <dependency groupId="org.apache.commons" artifactId="commons-lang3" version="3.1"/>
+          <dependency groupId="org.apache.commons" artifactId="commons-math3" version="3.2"/>
           <dependency groupId="com.googlecode.concurrentlinkedhashmap" artifactId="concurrentlinkedhashmap-lru" version="1.3"/>
           <dependency groupId="org.antlr" artifactId="antlr" version="3.2"/>
           <dependency groupId="org.slf4j" artifactId="slf4j-api" version="1.7.2"/>
@@ -453,6 +455,7 @@
         <dependency groupId="commons-cli" artifactId="commons-cli"/>
         <dependency groupId="commons-codec" artifactId="commons-codec"/>
         <dependency groupId="org.apache.commons" artifactId="commons-lang3"/>
+        <dependency groupId="org.apache.commons" artifactId="commons-math3"/>
         <dependency groupId="com.googlecode.concurrentlinkedhashmap" artifactId="concurrentlinkedhashmap-lru"/>
         <dependency groupId="org.antlr" artifactId="antlr"/>
         <dependency groupId="org.slf4j" artifactId="slf4j-api"/>
@@ -703,6 +706,9 @@
                     <fileset dir="${build.lib}">
                         <include name="**/*.jar" />
                     </fileset>
+                    <fileset dir="${build.tools.lib}">
+                        <include name="**/*.jar" />
+                    </fileset>
                 </path>
             </classpath>
         </javac>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/lib/commons-math3-3.2.jar
----------------------------------------------------------------------
diff --git a/lib/commons-math3-3.2.jar b/lib/commons-math3-3.2.jar
new file mode 100644
index 0000000..f8b7db2
Binary files /dev/null and b/lib/commons-math3-3.2.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/bin/cassandra-stressd
----------------------------------------------------------------------
diff --git a/tools/bin/cassandra-stressd b/tools/bin/cassandra-stressd
index 8d337e5..9110c5d 100755
--- a/tools/bin/cassandra-stressd
+++ b/tools/bin/cassandra-stressd
@@ -17,23 +17,25 @@
 # limitations under the License.
 
 DESC="Cassandra Stress Test Daemon"
+if [ "x$CASSANDRA_INCLUDE" = "x" ]; then 
+    for include in "`dirname $0`/cassandra.in.sh" \
+                   "$HOME/.cassandra.in.sh" \
+                   /usr/share/cassandra/cassandra.in.sh \
+                   /usr/local/share/cassandra/cassandra.in.sh \
+                   /opt/cassandra/cassandra.in.sh; do
+        if [ -r $include ]; then 
+            . $include
+            break   
+        fi      
+    done    
+elif [ -r $CASSANDRA_INCLUDE ]; then 
+    . $CASSANDRA_INCLUDE
+fi
 
-if [ "x$CLASSPATH" = "x" ]; then
-    
-    # execute from the build dir.
-    if [ -d `dirname $0`/../../build/classes ]; then
-        for directory in `dirname $0`/../../build/classes/*; do
-            CLASSPATH=$CLASSPATH:$directory
-        done
-    else
-        if [ -f `dirname $0`/../lib/stress.jar ]; then
-            CLASSPATH=`dirname $0`/../lib/stress.jar
-        fi
-    fi
-
-    for jar in `dirname $0`/../../lib/*.jar; do
-        CLASSPATH=$CLASSPATH:$jar
-    done
+if [ -x $JAVA_HOME/bin/java ]; then 
+    JAVA=$JAVA_HOME/bin/java
+else
+    JAVA=`which java`
 fi
 
 if [ -x $JAVA_HOME/bin/java ]; then

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/lib/cassandra-driver-core-2.0.0-rc2-SNAPSHOT-jar-with-dependencies.jar
----------------------------------------------------------------------
diff --git a/tools/lib/cassandra-driver-core-2.0.0-rc2-SNAPSHOT-jar-with-dependencies.jar b/tools/lib/cassandra-driver-core-2.0.0-rc2-SNAPSHOT-jar-with-dependencies.jar
new file mode 100644
index 0000000..1f4dafd
Binary files /dev/null and b/tools/lib/cassandra-driver-core-2.0.0-rc2-SNAPSHOT-jar-with-dependencies.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/lib/cassandra-driver-core-2.0.0-rc2-SNAPSHOT.jar
----------------------------------------------------------------------
diff --git a/tools/lib/cassandra-driver-core-2.0.0-rc2-SNAPSHOT.jar b/tools/lib/cassandra-driver-core-2.0.0-rc2-SNAPSHOT.jar
new file mode 100644
index 0000000..c0d4242
Binary files /dev/null and b/tools/lib/cassandra-driver-core-2.0.0-rc2-SNAPSHOT.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/Operation.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/Operation.java b/tools/stress/src/org/apache/cassandra/stress/Operation.java
new file mode 100644
index 0000000..fa7a453
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/Operation.java
@@ -0,0 +1,204 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.stress.generatedata.KeyGen;
+import org.apache.cassandra.stress.generatedata.RowGen;
+import org.apache.cassandra.stress.settings.Command;
+import org.apache.cassandra.stress.settings.CqlVersion;
+import org.apache.cassandra.stress.settings.SettingsCommandMixed;
+import org.apache.cassandra.stress.settings.StressSettings;
+import org.apache.cassandra.stress.util.JavaDriverClient;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.stress.util.Timer;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.transport.SimpleClient;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public abstract class Operation
+{
+    public final long index;
+    protected final State state;
+
+    public Operation(State state, long idx)
+    {
+        index = idx;
+        this.state = state;
+    }
+
+    public static interface RunOp
+    {
+        public boolean run() throws Exception;
+        public String key();
+        public int keyCount();
+    }
+
+    // one per thread!
+    public static final class State
+    {
+
+        public final StressSettings settings;
+        public final Timer timer;
+        public final Command type;
+        public final KeyGen keyGen;
+        public final RowGen rowGen;
+        public final List<ColumnParent> columnParents;
+        public final StressMetrics metrics;
+        public final SettingsCommandMixed.CommandSelector readWriteSelector;
+        private Object cqlCache;
+
+        public State(Command type, StressSettings settings, StressMetrics metrics)
+        {
+            this.type = type;
+            this.timer = metrics.getTiming().newTimer();
+            if (type == Command.MIXED)
+                readWriteSelector = ((SettingsCommandMixed) settings.command).selector();
+            else
+                readWriteSelector = null;
+            this.settings = settings;
+            this.keyGen = settings.keys.newKeyGen();
+            this.rowGen = settings.columns.newRowGen();
+            this.metrics = metrics;
+            if (!settings.columns.useSuperColumns)
+                columnParents = Collections.singletonList(new ColumnParent(settings.schema.columnFamily));
+            else
+            {
+                ColumnParent[] cp = new ColumnParent[settings.columns.superColumns];
+                for (int i = 0 ; i < cp.length ; i++)
+                    cp[i] = new ColumnParent("Super1").setSuper_column(ByteBufferUtil.bytes("S" + i));
+                columnParents = Arrays.asList(cp);
+            }
+        }
+        public boolean isCql3()
+        {
+            return settings.mode.cqlVersion == CqlVersion.CQL3;
+        }
+        public boolean isCql2()
+        {
+            return settings.mode.cqlVersion == CqlVersion.CQL2;
+        }
+        public Object getCqlCache()
+        {
+            return cqlCache;
+        }
+        public void storeCqlCache(Object val)
+        {
+            cqlCache = val;
+        }
+    }
+
+    protected ByteBuffer getKey()
+    {
+        return state.keyGen.getKeys(1, index).get(0);
+    }
+
+    protected List<ByteBuffer> getKeys(int count)
+    {
+        return state.keyGen.getKeys(count, index);
+    }
+
+    protected List<ByteBuffer> generateColumnValues()
+    {
+        return state.rowGen.generate(index);
+    }
+
+    /**
+     * Run operation
+     * @param client Cassandra Thrift client connection
+     * @throws IOException on any I/O error.
+     */
+    public abstract void run(ThriftClient client) throws IOException;
+
+    public void run(SimpleClient client) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    public void run(JavaDriverClient client) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    public void timeWithRetry(RunOp run) throws IOException
+    {
+        state.timer.start();
+
+        boolean success = false;
+        String exceptionMessage = null;
+
+        for (int t = 0; t < state.settings.command.tries; t++)
+        {
+            if (success)
+                break;
+
+            try
+            {
+                success = run.run();
+            }
+            catch (Exception e)
+            {
+                System.err.println(e);
+                exceptionMessage = getExceptionMessage(e);
+                success = false;
+            }
+        }
+
+        state.timer.stop(run.keyCount());
+
+        if (!success)
+        {
+            error(String.format("Operation [%d] retried %d times - error executing for key %s %s%n",
+                    index,
+                    state.settings.command.tries,
+                    run.key(),
+                    (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
+        }
+
+    }
+
+    protected String getExceptionMessage(Exception e)
+    {
+        String className = e.getClass().getSimpleName();
+        String message = (e instanceof InvalidRequestException) ? ((InvalidRequestException) e).getWhy() : e.getMessage();
+        return (message == null) ? "(" + className + ")" : String.format("(%s): %s", className, message);
+    }
+
+    protected void error(String message) throws IOException
+    {
+        if (!state.settings.command.ignoreErrors)
+            throw new IOException(message);
+        else
+            System.err.println(message);
+    }
+
+    public static ByteBuffer getColumnNameBytes(int i)
+    {
+        return ByteBufferUtil.bytes("C" + i);
+    }
+
+    public static String getColumnName(int i)
+    {
+        return "C" + i;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/Session.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/Session.java b/tools/stress/src/org/apache/cassandra/stress/Session.java
deleted file mode 100644
index 8d138f5..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/Session.java
+++ /dev/null
@@ -1,841 +0,0 @@
-/**
- * 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;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.cli.*;
-import org.apache.commons.lang3.StringUtils;
-
-import com.yammer.metrics.Metrics;
-
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.cli.transport.FramedTransportFactory;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.transport.SimpleClient;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportFactory;
-
-public class Session implements Serializable
-{
-    // command line options
-    public static final Options availableOptions = new Options();
-
-    public static final String KEYSPACE_NAME = "Keyspace1";
-    public static final String DEFAULT_COMPARATOR = "AsciiType";
-    public static final String DEFAULT_VALIDATOR  = "BytesType";
-
-    private static InetAddress localInetAddress;
-
-    public final AtomicInteger operations = new AtomicInteger();
-    public final AtomicInteger keys = new AtomicInteger();
-    public final com.yammer.metrics.core.Timer latency = Metrics.newTimer(Session.class, "latency");
-
-    private static final String SSL_TRUSTSTORE = "truststore";
-    private static final String SSL_TRUSTSTORE_PW = "truststore-password";
-    private static final String SSL_PROTOCOL = "ssl-protocol";
-    private static final String SSL_ALGORITHM = "ssl-alg";
-    private static final String SSL_STORE_TYPE = "store-type";
-    private static final String SSL_CIPHER_SUITES = "ssl-ciphers";
-
-    static
-    {
-        availableOptions.addOption("h",  "help",                 false,  "Show this help message and exit");
-        availableOptions.addOption("n",  "num-keys",             true,   "Number of keys, default:1000000");
-        availableOptions.addOption("F",  "num-different-keys",   true,   "Number of different keys (if < NUM-KEYS, the same key will re-used multiple times), default:NUM-KEYS");
-        availableOptions.addOption("N",  "skip-keys",            true,   "Fraction of keys to skip initially, default:0");
-        availableOptions.addOption("t",  "threads",              true,   "Number of threads to use, default:50");
-        availableOptions.addOption("c",  "cells",                true,   "Number of cells per key, default:5");
-        availableOptions.addOption("S",  "column-size",          true,   "Size of column values in bytes, default:34");
-        availableOptions.addOption("C",  "cardinality",          true,   "Number of unique values stored in cells, default:50");
-        availableOptions.addOption("d",  "nodes",                true,   "Host nodes (comma separated), default:locahost");
-        availableOptions.addOption("D",  "nodesfile",            true,   "File containing host nodes (one per line)");
-        availableOptions.addOption("s",  "stdev",                true,   "Standard Deviation Factor, default:0.1");
-        availableOptions.addOption("r",  "random",               false,  "Use random key generator (STDEV will have no effect), default:false");
-        availableOptions.addOption("f",  "file",                 true,   "Write output to given file");
-        availableOptions.addOption("p",  "port",                 true,   "Thrift port, default:9160");
-        availableOptions.addOption("o",  "operation",            true,   "Operation to perform (INSERT, READ, RANGE_SLICE, INDEXED_RANGE_SLICE, MULTI_GET, COUNTER_ADD, COUNTER_GET), default:INSERT");
-        availableOptions.addOption("u",  "supercolumns",         true,   "Number of super columns per key, default:1");
-        availableOptions.addOption("y",  "family-type",          true,   "Column Family Type (Super, Standard), default:Standard");
-        availableOptions.addOption("K",  "keep-trying",          true,   "Retry on-going operation N times (in case of failure). positive integer, default:10");
-        availableOptions.addOption("k",  "keep-going",           false,  "Ignore errors inserting or reading (when set, --keep-trying has no effect), default:false");
-        availableOptions.addOption("i",  "progress-interval",    true,   "Progress Report Interval (seconds), default:10");
-        availableOptions.addOption("g",  "keys-per-call",        true,   "Number of keys to get_range_slices or multiget per call, default:1000");
-        availableOptions.addOption("l",  "replication-factor",   true,   "Replication Factor to use when creating needed column families, default:1");
-        availableOptions.addOption("L",  "enable-cql",           false,  "Perform queries using CQL2 (Cassandra Query Language v 2.0.0)");
-        availableOptions.addOption("L3", "enable-cql3",          false,  "Perform queries using CQL3 (Cassandra Query Language v 3.0.0)");
-        availableOptions.addOption("b",  "enable-native-protocol",  false,  "Use the binary native protocol (only work along with -L3)");
-        availableOptions.addOption("P",  "use-prepared-statements", false, "Perform queries using prepared statements (only applicable to CQL).");
-        availableOptions.addOption("e",  "consistency-level",    true,   "Consistency Level to use (ONE, QUORUM, LOCAL_QUORUM, EACH_QUORUM, ALL, ANY), default:ONE");
-        availableOptions.addOption("x",  "create-index",         true,   "Type of index to create on needed column families (KEYS)");
-        availableOptions.addOption("R",  "replication-strategy", true,   "Replication strategy to use (only on insert if keyspace does not exist), default:org.apache.cassandra.locator.SimpleStrategy");
-        availableOptions.addOption("O",  "strategy-properties",  true,   "Replication strategy properties in the following format <dc_name>:<num>,<dc_name>:<num>,...");
-        availableOptions.addOption("W",  "no-replicate-on-write",false,  "Set replicate_on_write to false for counters. Only counter add with CL=ONE will work");
-        availableOptions.addOption("V",  "average-size-values",  false,  "Generate column values of average rather than specific size");
-        availableOptions.addOption("T",  "send-to",              true,   "Send this as a request to the stress daemon at specified address.");
-        availableOptions.addOption("I",  "compression",          true,   "Specify the compression to use for sstable, default:no compression");
-        availableOptions.addOption("Q",  "query-names",          true,   "Comma-separated list of column names to retrieve from each row.");
-        availableOptions.addOption("Z",  "compaction-strategy",  true,   "CompactionStrategy to use.");
-        availableOptions.addOption("U",  "comparator",           true,   "Cell Comparator to use. Currently supported types are: TimeUUIDType, AsciiType, UTF8Type.");
-        availableOptions.addOption("tf", "transport-factory",    true,   "Fully-qualified TTransportFactory class name for creating a connection. Note: For Thrift over SSL, use org.apache.cassandra.stress.SSLTransportFactory.");
-        availableOptions.addOption("ns", "no-statistics",        false,  "Turn off the aggegate statistics that is normally output after completion.");
-        availableOptions.addOption("ts", SSL_TRUSTSTORE,         true, "SSL: full path to truststore");
-        availableOptions.addOption("tspw", SSL_TRUSTSTORE_PW,    true, "SSL: full path to truststore");
-        availableOptions.addOption("prtcl", SSL_PROTOCOL,        true, "SSL: connections protocol to use (default: TLS)");
-        availableOptions.addOption("alg", SSL_ALGORITHM,         true, "SSL: algorithm (default: SunX509)");
-        availableOptions.addOption("st", SSL_STORE_TYPE,         true, "SSL: type of store");
-        availableOptions.addOption("ciphers", SSL_CIPHER_SUITES, true, "SSL: comma-separated list of encryption suites to use");
-        availableOptions.addOption("th", "throttle",             true, "Throttle the total number of operations per second to a maximum amount.");
-        availableOptions.addOption("un", "username",             true, "Username for authentication.");
-        availableOptions.addOption("pw", "password",             true, "Password for authentication.");
-    }
-
-    private int numKeys          = 1000 * 1000;
-    private int numDifferentKeys = numKeys;
-    private float skipKeys       = 0;
-    private int threads          = 50;
-    private int columns          = 5;
-    private int columnSize       = 34;
-    private int cardinality      = 50;
-    public String[] nodes        = new String[] { "127.0.0.1" };
-    private boolean random       = false;
-    private int retryTimes       = 10;
-    public int port              = 9160;
-    private int superColumns     = 1;
-    private String compression   = null;
-    private String compactionStrategy = null;
-    private String username      = null;
-    private String password      = null;
-
-    private int progressInterval  = 10;
-    private int keysPerCall       = 1000;
-    private boolean replicateOnWrite = true;
-    private boolean ignoreErrors  = false;
-    private boolean enable_cql    = false;
-    private boolean use_prepared  = false;
-    private boolean trace         = false;
-    private boolean captureStatistics = true;
-    public boolean use_native_protocol = false;
-    private double maxOpsPerSecond = Double.MAX_VALUE;
-
-    private final String outFileName;
-
-    private IndexType indexType = null;
-    private Stress.Operations operation = Stress.Operations.INSERT;
-    private ColumnFamilyType columnFamilyType = ColumnFamilyType.Standard;
-    private ConsistencyLevel consistencyLevel = ConsistencyLevel.ONE;
-    private String replicationStrategy = "org.apache.cassandra.locator.SimpleStrategy";
-    private Map<String, String> replicationStrategyOptions = new HashMap<String, String>();
-
-    // if we know exactly column names that we want to read (set by -Q option)
-    public final List<ByteBuffer> columnNames;
-
-    public String cqlVersion;
-
-    public final boolean averageSizeValues;
-
-    // required by Gaussian distribution.
-    protected int   mean;
-    protected float sigma;
-
-    public final InetAddress sendToDaemon;
-    public final String comparator;
-    public final boolean timeUUIDComparator;
-    public double traceProbability = 0.0;
-    public EncryptionOptions encOptions = new ClientEncryptionOptions();
-    public TTransportFactory transportFactory = new FramedTransportFactory();
-
-    public Session(String[] arguments) throws IllegalArgumentException, SyntaxException
-    {
-        float STDev = 0.1f;
-        CommandLineParser parser = new PosixParser();
-
-        try
-        {
-            CommandLine cmd = parser.parse(availableOptions, arguments);
-
-            if (cmd.getArgs().length > 0)
-            {
-                System.err.println("Application does not allow arbitrary arguments: " + StringUtils.join(cmd.getArgList(), ", "));
-                System.exit(1);
-            }
-
-            if (cmd.hasOption("h"))
-                throw new IllegalArgumentException("help");
-
-            if (cmd.hasOption("n"))
-                numKeys = Integer.parseInt(cmd.getOptionValue("n"));
-
-            if (cmd.hasOption("F"))
-                numDifferentKeys = Integer.parseInt(cmd.getOptionValue("F"));
-            else
-                numDifferentKeys = numKeys;
-
-            if (cmd.hasOption("N"))
-                skipKeys = Float.parseFloat(cmd.getOptionValue("N"));
-
-            if (cmd.hasOption("t"))
-                threads = Integer.parseInt(cmd.getOptionValue("t"));
-
-            if (cmd.hasOption("c"))
-                columns = Integer.parseInt(cmd.getOptionValue("c"));
-
-            if (cmd.hasOption("S"))
-                columnSize = Integer.parseInt(cmd.getOptionValue("S"));
-
-            if (cmd.hasOption("C"))
-                cardinality = Integer.parseInt(cmd.getOptionValue("C"));
-
-            if (cmd.hasOption("d"))
-                nodes = cmd.getOptionValue("d").split(",");
-
-            if (cmd.hasOption("D"))
-            {
-                try
-                {
-                    String node;
-                    List<String> tmpNodes = new ArrayList<String>();
-                    BufferedReader in = new BufferedReader(new InputStreamReader(new FileInputStream(cmd.getOptionValue("D"))));
-                    try
-                    {
-                        while ((node = in.readLine()) != null)
-                        {
-                            if (node.length() > 0)
-                                tmpNodes.add(node);
-                        }
-                        nodes = tmpNodes.toArray(new String[tmpNodes.size()]);
-                    }
-                    finally
-                    {
-                        in.close();
-                    }
-                }
-                catch(IOException ioe)
-                {
-                    throw new RuntimeException(ioe);
-                }
-            }
-
-            if (cmd.hasOption("s"))
-                STDev = Float.parseFloat(cmd.getOptionValue("s"));
-
-            if (cmd.hasOption("r"))
-                random = true;
-
-            outFileName = (cmd.hasOption("f")) ? cmd.getOptionValue("f") : null;
-
-            if (cmd.hasOption("p"))
-                port = Integer.parseInt(cmd.getOptionValue("p"));
-
-            if (cmd.hasOption("o"))
-                operation = Stress.Operations.valueOf(cmd.getOptionValue("o").toUpperCase());
-
-            if (cmd.hasOption("u"))
-                superColumns = Integer.parseInt(cmd.getOptionValue("u"));
-
-            if (cmd.hasOption("y"))
-                columnFamilyType = ColumnFamilyType.valueOf(cmd.getOptionValue("y"));
-
-            if (cmd.hasOption("K"))
-            {
-                retryTimes = Integer.valueOf(cmd.getOptionValue("K"));
-
-                if (retryTimes <= 0)
-                {
-                    throw new RuntimeException("--keep-trying option value should be > 0");
-                }
-            }
-
-            if (cmd.hasOption("k"))
-            {
-                retryTimes = 1;
-                ignoreErrors = true;
-            }
-
-
-            if (cmd.hasOption("i"))
-                progressInterval = Integer.parseInt(cmd.getOptionValue("i"));
-
-            if (cmd.hasOption("g"))
-                keysPerCall = Integer.parseInt(cmd.getOptionValue("g"));
-
-            if (cmd.hasOption("th"))
-                maxOpsPerSecond = Double.parseDouble(cmd.getOptionValue("th"));
-
-            if (cmd.hasOption("e"))
-                consistencyLevel = ConsistencyLevel.valueOf(cmd.getOptionValue("e").toUpperCase());
-
-            if (cmd.hasOption("x"))
-                indexType = IndexType.valueOf(cmd.getOptionValue("x").toUpperCase());
-
-            if (cmd.hasOption("R"))
-                replicationStrategy = cmd.getOptionValue("R");
-
-            if (cmd.hasOption("l"))
-                replicationStrategyOptions.put("replication_factor", String.valueOf(Integer.parseInt(cmd.getOptionValue("l"))));
-            else if (replicationStrategy.endsWith("SimpleStrategy"))
-                replicationStrategyOptions.put("replication_factor", "1");
-
-            if (cmd.hasOption("L"))
-            {
-                enable_cql = true;
-                cqlVersion = "2.0.0";
-            }
-
-            if (cmd.hasOption("L3"))
-            {
-                enable_cql = true;
-                cqlVersion = "3.0.0";
-            }
-
-            if (cmd.hasOption("b"))
-            {
-                if (!(enable_cql && cqlVersion.startsWith("3")))
-                    throw new IllegalArgumentException("Cannot use binary protocol without -L3");
-                use_native_protocol = true;
-            }
-
-            if (cmd.hasOption("P"))
-            {
-                if (!enable_cql)
-                {
-                    System.err.println("-P/--use-prepared-statements is only applicable with CQL (-L/--enable-cql)");
-                    System.exit(-1);
-                }
-                use_prepared = true;
-            }
-
-            if (cmd.hasOption("O"))
-            {
-                String[] pairs = StringUtils.split(cmd.getOptionValue("O"), ',');
-
-                for (String pair : pairs)
-                {
-                    String[] keyAndValue = StringUtils.split(pair, ':');
-
-                    if (keyAndValue.length != 2)
-                        throw new RuntimeException("Invalid --strategy-properties value.");
-
-                    replicationStrategyOptions.put(keyAndValue[0], keyAndValue[1]);
-                }
-            }
-
-            if (cmd.hasOption("W"))
-                replicateOnWrite = false;
-
-            if (cmd.hasOption("I"))
-                compression = cmd.getOptionValue("I");
-
-            averageSizeValues = cmd.hasOption("V");
-
-            try
-            {
-                sendToDaemon = cmd.hasOption("send-to")
-                                ? InetAddress.getByName(cmd.getOptionValue("send-to"))
-                                : null;
-            }
-            catch (UnknownHostException e)
-            {
-                throw new RuntimeException(e);
-            }
-
-            if (cmd.hasOption("Q"))
-            {
-                AbstractType comparator = TypeParser.parse(DEFAULT_COMPARATOR);
-
-                String[] names = StringUtils.split(cmd.getOptionValue("Q"), ",");
-                columnNames = new ArrayList<ByteBuffer>(names.length);
-
-                for (String columnName : names)
-                    columnNames.add(comparator.fromString(columnName));
-            }
-            else
-            {
-                columnNames = null;
-            }
-
-            if (cmd.hasOption("Z"))
-            {
-                compactionStrategy = cmd.getOptionValue("Z");
-
-                try
-                {
-                    // validate compaction strategy class
-                    CFMetaData.createCompactionStrategy(compactionStrategy);
-                }
-                catch (ConfigurationException e)
-                {
-                    System.err.println(e.getMessage());
-                    System.exit(1);
-                }
-            }
-
-            if (cmd.hasOption("U"))
-            {
-                AbstractType parsed = null;
-
-                try
-                {
-                    parsed = TypeParser.parse(cmd.getOptionValue("U"));
-                }
-                catch (ConfigurationException e)
-                {
-                    System.err.println(e.getMessage());
-                    System.exit(1);
-                }
-
-                comparator = cmd.getOptionValue("U");
-                timeUUIDComparator = parsed instanceof TimeUUIDType;
-
-                if (!(parsed instanceof TimeUUIDType || parsed instanceof AsciiType || parsed instanceof UTF8Type))
-                {
-                    System.err.println("Currently supported types are: TimeUUIDType, AsciiType, UTF8Type.");
-                    System.exit(1);
-                }
-            }
-            else
-            {
-                comparator = null;
-                timeUUIDComparator = false;
-            }
-
-            if (cmd.hasOption("ns"))
-            {
-                captureStatistics = false;
-            }
-
-            if(cmd.hasOption(SSL_TRUSTSTORE))
-                encOptions.truststore = cmd.getOptionValue(SSL_TRUSTSTORE);
-
-            if(cmd.hasOption(SSL_TRUSTSTORE_PW))
-                encOptions.truststore_password = cmd.getOptionValue(SSL_TRUSTSTORE_PW);
-
-            if(cmd.hasOption(SSL_PROTOCOL))
-                encOptions.protocol = cmd.getOptionValue(SSL_PROTOCOL);
-
-            if(cmd.hasOption(SSL_ALGORITHM))
-                encOptions.algorithm = cmd.getOptionValue(SSL_ALGORITHM);
-
-            if(cmd.hasOption(SSL_STORE_TYPE))
-                encOptions.store_type = cmd.getOptionValue(SSL_STORE_TYPE);
-
-            if(cmd.hasOption(SSL_CIPHER_SUITES))
-                encOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(",");
-
-            if (cmd.hasOption("tf"))
-                transportFactory = validateAndSetTransportFactory(cmd.getOptionValue("tf"));
-
-            if (cmd.hasOption("un"))
-                username = cmd.getOptionValue("un");
-
-            if (cmd.hasOption("pw"))
-                password = cmd.getOptionValue("pw");
-        }
-        catch (ParseException e)
-        {
-            throw new IllegalArgumentException(e.getMessage(), e);
-        }
-        catch (ConfigurationException e)
-        {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-
-        mean  = numDifferentKeys / 2;
-        sigma = numDifferentKeys * STDev;
-    }
-
-    private TTransportFactory validateAndSetTransportFactory(String transportFactory)
-    {
-        try
-        {
-            Class factory = Class.forName(transportFactory);
-
-            if(!TTransportFactory.class.isAssignableFrom(factory))
-                throw new IllegalArgumentException(String.format("transport factory '%s' " +
-                        "not derived from TTransportFactory", transportFactory));
-
-            return (TTransportFactory) factory.newInstance();
-        }
-        catch (Exception e)
-        {
-            throw new IllegalArgumentException(String.format("Cannot create a transport factory '%s'.", transportFactory), e);
-        }
-    }
-
-    public int getCardinality()
-    {
-        return cardinality;
-    }
-
-    public int getColumnSize()
-    {
-        return columnSize;
-    }
-
-    public int getColumnsPerKey()
-    {
-        return columns;
-    }
-
-    public ColumnFamilyType getColumnFamilyType()
-    {
-        return columnFamilyType;
-    }
-
-    public int getNumKeys()
-    {
-        return numKeys;
-    }
-
-    public int getNumDifferentKeys()
-    {
-        return numDifferentKeys;
-    }
-
-    public int getThreads()
-    {
-        return threads;
-    }
-
-    public double getMaxOpsPerSecond()
-    {
-        return maxOpsPerSecond;
-    }
-
-    public float getSkipKeys()
-    {
-        return skipKeys;
-    }
-
-    public int getSuperColumns()
-    {
-        return superColumns;
-    }
-
-    public int getKeysPerThread()
-    {
-        return numKeys / threads;
-    }
-
-    public int getTotalKeysLength()
-    {
-        return Integer.toString(numDifferentKeys).length();
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return consistencyLevel;
-    }
-
-    public int getRetryTimes()
-    {
-        return retryTimes;
-    }
-
-    public boolean ignoreErrors()
-    {
-        return ignoreErrors;
-    }
-
-    public Stress.Operations getOperation()
-    {
-        return operation;
-    }
-
-    public PrintStream getOutputStream()
-    {
-        try
-        {
-            return (outFileName == null) ? System.out : new PrintStream(new FileOutputStream(outFileName));
-        }
-        catch (FileNotFoundException e)
-        {
-            throw new RuntimeException(e.getMessage(), e);
-        }
-    }
-
-    public int getProgressInterval()
-    {
-        return progressInterval;
-    }
-
-    public boolean useRandomGenerator()
-    {
-        return random;
-    }
-
-    public int getKeysPerCall()
-    {
-        return keysPerCall;
-    }
-
-    // required by Gaussian distribution
-    public int getMean()
-    {
-        return mean;
-    }
-
-    // required by Gaussian distribution
-    public float getSigma()
-    {
-        return sigma;
-    }
-
-    public boolean isCQL()
-    {
-        return enable_cql;
-    }
-
-    public boolean usePreparedStatements()
-    {
-        return use_prepared;
-    }
-
-    public boolean outputStatistics()
-    {
-        return captureStatistics;
-    }
-
-    /**
-     * Create Keyspace with Standard and Super/Counter column families
-     */
-    public void createKeySpaces()
-    {
-        KsDef keyspace = new KsDef();
-        String defaultComparator = comparator == null ? DEFAULT_COMPARATOR : comparator;
-
-        // column family for standard columns
-        CfDef standardCfDef = new CfDef(KEYSPACE_NAME, "Standard1");
-        Map<String, String> compressionOptions = new HashMap<String, String>();
-        if (compression != null)
-            compressionOptions.put("sstable_compression", compression);
-
-        standardCfDef.setComparator_type(defaultComparator)
-                     .setDefault_validation_class(DEFAULT_VALIDATOR)
-                     .setCompression_options(compressionOptions);
-
-        if (!timeUUIDComparator)
-        {
-            for (int i = 0; i < getColumnsPerKey(); 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_NAME, "Super1").setColumn_type("Super");
-        superCfDef.setComparator_type(DEFAULT_COMPARATOR)
-                  .setSubcomparator_type(defaultComparator)
-                  .setDefault_validation_class(DEFAULT_VALIDATOR)
-                  .setCompression_options(compressionOptions);
-
-        // column family for standard counters
-        CfDef counterCfDef = new CfDef(KEYSPACE_NAME, "Counter1").setComparator_type(defaultComparator)
-                                                                 .setComparator_type(defaultComparator)
-                                                                 .setDefault_validation_class("CounterColumnType")
-                                                                 .setReplicate_on_write(replicateOnWrite)
-                                                                 .setCompression_options(compressionOptions);
-
-        // column family with counter super columns
-        CfDef counterSuperCfDef = new CfDef(KEYSPACE_NAME, "SuperCounter1").setComparator_type(defaultComparator)
-                                                                           .setDefault_validation_class("CounterColumnType")
-                                                                           .setReplicate_on_write(replicateOnWrite)
-                                                                           .setColumn_type("Super")
-                                                                           .setCompression_options(compressionOptions);
-
-        keyspace.setName(KEYSPACE_NAME);
-        keyspace.setStrategy_class(replicationStrategy);
-
-        if (!replicationStrategyOptions.isEmpty())
-        {
-            keyspace.setStrategy_options(replicationStrategyOptions);
-        }
-
-        if (compactionStrategy != null)
-        {
-            standardCfDef.setCompaction_strategy(compactionStrategy);
-            superCfDef.setCompaction_strategy(compactionStrategy);
-            counterCfDef.setCompaction_strategy(compactionStrategy);
-            counterSuperCfDef.setCompaction_strategy(compactionStrategy);
-        }
-
-        keyspace.setCf_defs(new ArrayList<CfDef>(Arrays.asList(standardCfDef, superCfDef, counterCfDef, counterSuperCfDef)));
-
-        CassandraClient client = getClient(false);
-
-        try
-        {
-            client.system_add_keyspace(keyspace);
-
-            /* CQL3 counter cf */
-            client.set_cql_version("3.0.0"); // just to create counter cf for cql3
-
-            client.set_keyspace(KEYSPACE_NAME);
-            client.execute_cql3_query(createCounterCFStatementForCQL3(), Compression.NONE, ConsistencyLevel.ONE);
-
-            if (enable_cql)
-                client.set_cql_version(cqlVersion);
-            /* end */
-
-            System.out.println(String.format("Created keyspaces. Sleeping %ss for propagation.", nodes.length));
-            Thread.sleep(nodes.length * 1000); // seconds
-        }
-        catch (InvalidRequestException e)
-        {
-            System.err.println("Unable to create stress keyspace: " + e.getWhy());
-        }
-        catch (Exception e)
-        {
-            System.err.println(e.getMessage());
-        }
-    }
-
-    /**
-     * Thrift client connection with Keyspace1 set.
-     * @return cassandra client connection
-     */
-    public CassandraClient getClient()
-    {
-        return getClient(true);
-    }
-
-    /**
-     * Thrift client connection
-     * @param setKeyspace - should we set keyspace for client or not
-     * @return cassandra client connection
-     */
-    public CassandraClient getClient(boolean setKeyspace)
-    {
-        // random node selection for fake load balancing
-        String currentNode = nodes[Stress.randomizer.nextInt(nodes.length)];
-
-        TSocket socket = new TSocket(currentNode, port);
-        TTransport transport = transportFactory.getTransport(socket);
-        CassandraClient client = new CassandraClient(new TBinaryProtocol(transport));
-
-        try
-        {
-            if (!transport.isOpen())
-                transport.open();
-
-            if (enable_cql)
-                client.set_cql_version(cqlVersion);
-
-            if (setKeyspace)
-                client.set_keyspace("Keyspace1");
-
-            if (username != null && password != null)
-            {
-                Map<String, String> credentials = new HashMap<String, String>();
-                credentials.put(IAuthenticator.USERNAME_KEY, username);
-                credentials.put(IAuthenticator.PASSWORD_KEY, password);
-                AuthenticationRequest authenticationRequest = new AuthenticationRequest(credentials);
-                client.login(authenticationRequest);
-            }
-        }
-        catch (AuthenticationException e)
-        {
-            throw new RuntimeException(e.getWhy());
-        }
-        catch (AuthorizationException e)
-        {
-            throw new RuntimeException(e.getWhy());
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e.getWhy());
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e.getMessage());
-        }
-
-        return client;
-    }
-
-    public SimpleClient getNativeClient()
-    {
-        try
-        {
-            String currentNode = nodes[Stress.randomizer.nextInt(nodes.length)];
-            SimpleClient client = new SimpleClient(currentNode, 9042);
-            client.connect(false);
-            client.execute("USE \"Keyspace1\";", org.apache.cassandra.db.ConsistencyLevel.ONE);
-            return client;
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e.getMessage());
-        }
-    }
-
-    public static InetAddress getLocalAddress()
-    {
-        if (localInetAddress == null)
-        {
-            try
-            {
-                localInetAddress = InetAddress.getLocalHost();
-            }
-            catch (UnknownHostException e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        return localInetAddress;
-    }
-
-    private ByteBuffer createCounterCFStatementForCQL3()
-    {
-        StringBuilder counter3 = new StringBuilder("CREATE TABLE \"Counter3\" (KEY blob PRIMARY KEY, ");
-
-        for (int i = 0; i < getColumnsPerKey(); i++)
-        {
-            counter3.append("c").append(i).append(" counter");
-            if (i != getColumnsPerKey() - 1)
-                counter3.append(", ");
-        }
-        counter3.append(");");
-
-        return ByteBufferUtil.bytes(counter3.toString());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/Stress.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/Stress.java b/tools/stress/src/org/apache/cassandra/stress/Stress.java
index 738a1c0..38af4f6 100644
--- a/tools/stress/src/org/apache/cassandra/stress/Stress.java
+++ b/tools/stress/src/org/apache/cassandra/stress/Stress.java
@@ -17,48 +17,65 @@
  */
 package org.apache.cassandra.stress;
 
-import org.apache.commons.cli.Option;
-
 import java.io.*;
 import java.net.Socket;
 import java.net.SocketException;
-import java.util.Random;
+
+import org.apache.cassandra.stress.settings.StressSettings;
 
 public final class Stress
 {
-    public static enum Operations
-    {
-        INSERT, READ, RANGE_SLICE, INDEXED_RANGE_SLICE, MULTI_GET, COUNTER_ADD, COUNTER_GET
-    }
 
-    public static Session session;
-    public static Random randomizer = new Random();
+    /**
+     * Known issues:
+     * - uncertainty/stderr assumes op-rates are normally distributed. Due to GC (and possibly latency stepping from
+     * different media, though the variance of request ratio across media should be normally distributed), they are not.
+     * Should attempt to account for pauses in stderr calculation, possibly by assuming these pauses are a separate
+     * normally distributed occurrence
+     * - Under very mixed work loads, the uncertainty calculations and op/s reporting really don't mean much. Should
+     * consider breaking op/s down per workload, or should have a lower-bound on inspection interval based on clustering
+     * of operations and thread count.
+     *
+     *
+     * Future improvements:
+     * - Configurable connection compression
+     * - Java driver support
+     * - Per column data generators
+     * - Automatic column/schema detection if provided with a CF
+     * - target rate produces a very steady work rate, and if we want to simulate a real op rate for an
+     *   application we should have some variation in the actual op rate within any time-slice.
+     * - auto rate should vary the thread count based on performance improvement, potentially starting on a very low
+     *   thread count with a high error rate / low count to get some basic numbers
+     */
+
     private static volatile boolean stopped = false;
 
     public static void main(String[] arguments) throws Exception
     {
+        final StressSettings settings;
         try
         {
-            session = new Session(arguments);
+            settings = StressSettings.parse(arguments);
         }
         catch (IllegalArgumentException e)
         {
             printHelpMessage();
+            e.printStackTrace();
             return;
         }
 
-        PrintStream outStream = session.getOutputStream();
+        PrintStream logout = settings.log.getOutput();
 
-        if (session.sendToDaemon != null)
+        if (settings.sendToDaemon != null)
         {
-            Socket socket = new Socket(session.sendToDaemon, 2159);
+            Socket socket = new Socket(settings.sendToDaemon, 2159);
 
             ObjectOutputStream out = new ObjectOutputStream(socket.getOutputStream());
             BufferedReader inp = new BufferedReader(new InputStreamReader(socket.getInputStream()));
 
             Runtime.getRuntime().addShutdownHook(new ShutDown(socket, out));
 
-            out.writeObject(session);
+            out.writeObject(settings);
 
             String line;
 
@@ -72,7 +89,7 @@ public final class Stress
                         break;
                     }
 
-                    outStream.println(line);
+                    logout.println(line);
                 }
             }
             catch (SocketException e)
@@ -88,10 +105,8 @@ public final class Stress
         }
         else
         {
-            StressAction stressAction = new StressAction(session, outStream);
-            stressAction.start();
-            stressAction.join();
-            System.exit(stressAction.getReturnCode());
+            StressAction stressAction = new StressAction(settings, logout);
+            stressAction.run();
         }
     }
 
@@ -100,15 +115,7 @@ public final class Stress
      */
     public static void printHelpMessage()
     {
-        System.out.println("Usage: ./bin/cassandra-stress [options]\n\nOptions:");
-
-        for(Object o : Session.availableOptions.getOptions())
-        {
-            Option option = (Option) o;
-            String upperCaseName = option.getLongOpt().toUpperCase();
-            System.out.println(String.format("-%s%s, --%s%s%n\t\t%s%n", option.getOpt(), (option.hasArg()) ? " "+upperCaseName : "",
-                                                            option.getLongOpt(), (option.hasArg()) ? "="+upperCaseName : "", option.getDescription()));
-        }
+        StressSettings.printHelp();
     }
 
     private static class ShutDown extends Thread

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/StressAction.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressAction.java b/tools/stress/src/org/apache/cassandra/stress/StressAction.java
index 7098d0b..0312093 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressAction.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressAction.java
@@ -17,322 +17,527 @@
  */
 package org.apache.cassandra.stress;
 
+import java.io.IOException;
+import java.io.OutputStream;
 import java.io.PrintStream;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.SynchronousQueue;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
-import com.google.common.util.concurrent.Uninterruptibles;
 import com.google.common.util.concurrent.RateLimiter;
-import com.yammer.metrics.stats.Snapshot;
+import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.cassandra.stress.operations.*;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
+import org.apache.cassandra.stress.settings.*;
+import org.apache.cassandra.stress.util.JavaDriverClient;
+import org.apache.cassandra.stress.util.ThriftClient;
 import org.apache.cassandra.transport.SimpleClient;
 
-public class StressAction extends Thread
+public class StressAction implements Runnable
 {
-    /**
-     * Producer-Consumer model: 1 producer, N consumers
-     */
-    private final BlockingQueue<Operation> operations = new SynchronousQueue<Operation>(true);
 
-    private final Session client;
+    private final StressSettings settings;
     private final PrintStream output;
 
-    private volatile boolean stop = false;
-
-    public static final int SUCCESS = 0;
-    public static final int FAILURE = 1;
-
-    private volatile int returnCode = -1;
-
-    public StressAction(Session session, PrintStream out)
+    public StressAction(StressSettings settings, PrintStream out)
     {
-        client = session;
+        this.settings = settings;
         output = out;
     }
 
     public void run()
     {
-        Snapshot latency;
-        long oldLatency;
-        int epoch, total, oldTotal, keyCount, oldKeyCount;
-
         // creating keyspace and column families
-        if (client.getOperation() == Stress.Operations.INSERT || client.getOperation() == Stress.Operations.COUNTER_ADD)
-            client.createKeySpaces();
+        settings.maybeCreateKeyspaces();
 
-        int threadCount = client.getThreads();
-        Consumer[] consumers = new Consumer[threadCount];
+        warmup(settings.command.type, settings.command);
 
-        output.println("total,interval_op_rate,interval_key_rate,latency,95th,99.9th,elapsed_time");
+        output.println("Sleeping 2s...");
+        Uninterruptibles.sleepUninterruptibly(2, TimeUnit.SECONDS);
 
-        int itemsPerThread = client.getKeysPerThread();
-        int modulo = client.getNumKeys() % threadCount;
-        RateLimiter rateLimiter = RateLimiter.create(client.getMaxOpsPerSecond());
+        boolean success;
+        if (settings.rate.auto)
+            success = runAuto();
+        else
+            success = null != run(settings.command.type, settings.rate.threadCount, settings.command.count, output);
 
-        // creating required type of the threads for the test
-        for (int i = 0; i < threadCount; i++) {
-            if (i == threadCount - 1)
-                itemsPerThread += modulo; // last one is going to handle N + modulo items
+        if (success)
+            output.println("END");
+        else
+            output.println("FAILURE");
 
-            consumers[i] = new Consumer(itemsPerThread, rateLimiter);
-        }
+        settings.disconnect();
+    }
 
-        Producer producer = new Producer();
-        producer.start();
+    // type provided separately to support recursive call for mixed command with each command type it is performing
+    private void warmup(Command type, SettingsCommand command)
+    {
+        // warmup - do 50k iterations; by default hotspot compiles methods after 10k invocations
+        PrintStream warmupOutput = new PrintStream(new OutputStream() { @Override public void write(int b) throws IOException { } } );
+        int iterations;
+        switch (type.category)
+        {
+            case BASIC:
+                iterations = 50000;
+                break;
+            case MIXED:
+                for (Command subtype : ((SettingsCommandMixed) command).getCommands())
+                    warmup(subtype, command);
+                return;
+            case MULTI:
+                int keysAtOnce = ((SettingsCommandMulti) command).keysAtOnce;
+                iterations = Math.min(50000, (int) Math.ceil(500000d / keysAtOnce));
+                break;
+            default:
+                throw new IllegalStateException();
+        }
+        output.println(String.format("Warming up %s with %d iterations...", type, iterations));
+        run(type, 20, iterations, warmupOutput);
+    }
 
-        // starting worker threads
-        for (int i = 0; i < threadCount; i++)
-            consumers[i].start();
+    // TODO : permit varying more than just thread count
+    // TODO : vary thread count based on percentage improvement of previous increment, not by fixed amounts
+    private boolean runAuto()
+    {
+        int prevThreadCount = -1;
+        int threadCount = settings.rate.minAutoThreads;
+        List<StressMetrics> results = new ArrayList<>();
+        List<String> runIds = new ArrayList<>();
+        do
+        {
+            output.println(String.format("Running with %d threadCount", threadCount));
 
-        // initialization of the values
-        boolean terminate = false;
-        epoch = total = keyCount = 0;
+            StressMetrics result = run(settings.command.type, threadCount, settings.command.count, output);
+            if (result == null)
+                return false;
+            results.add(result);
 
-        int interval = client.getProgressInterval();
-        int epochIntervals = client.getProgressInterval() * 10;
-        long testStartTime = System.nanoTime();
-        
-        StressStatistics stats = new StressStatistics(client, output);
+            if (prevThreadCount > 0)
+                System.out.println(String.format("Improvement over %d threadCount: %.0f%%",
+                        prevThreadCount, 100 * averageImprovement(results, 1)));
 
-        while (!terminate)
-        {
-            if (stop)
-            {
-                producer.stopProducer();
-
-                for (Consumer consumer : consumers)
-                    consumer.stopConsume();
+            runIds.add(threadCount + " threadCount");
+            prevThreadCount = threadCount;
+            if (threadCount < 16)
+                threadCount *= 2;
+            else
+                threadCount *= 1.5;
 
+            if (!results.isEmpty() && threadCount > settings.rate.maxAutoThreads)
                 break;
+
+            if (settings.command.type.updates)
+            {
+                // pause an arbitrary period of time to let the commit log flush, etc. shouldn't make much difference
+                // as we only increase load, never decrease it
+                output.println("Sleeping for 15s");
+                try
+                {
+                    Thread.sleep(15 * 1000);
+                } catch (InterruptedException e)
+                {
+                    return false;
+                }
             }
+            // run until we have not improved throughput significantly for previous three runs
+        } while (hasAverageImprovement(results, 3, 0) && hasAverageImprovement(results, 5, settings.command.targetUncertainty));
 
-            Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+        // summarise all results
+        StressMetrics.summarise(runIds, results, output);
+        return true;
+    }
 
-            int alive = 0;
-            for (Thread thread : consumers)
-                if (thread.isAlive()) alive++;
+    private boolean hasAverageImprovement(List<StressMetrics> results, int count, double minImprovement)
+    {
+        if (results.size() < count + 1)
+            return true;
+        return averageImprovement(results, count) >= minImprovement;
+    }
 
-            if (alive == 0)
-                terminate = true;
+    private double averageImprovement(List<StressMetrics> results, int count)
+    {
+        double improvement = 0;
+        for (int i = results.size() - count ; i < results.size() ; i++)
+        {
+            double prev = results.get(i - 1).getTiming().getHistory().realOpRate();
+            double cur = results.get(i).getTiming().getHistory().realOpRate();
+            improvement += (cur - prev) / prev;
+        }
+        return improvement / count;
+    }
 
-            epoch++;
+    private StressMetrics run(Command type, int threadCount, long opCount, PrintStream output)
+    {
 
-            if (terminate || epoch > epochIntervals)
-            {
-                epoch = 0;
+        output.println(String.format("Running %s with %d threads %s",
+                type.toString(),
+                threadCount,
+                opCount > 0 ? " for " + opCount + " iterations" : "until stderr of mean < " + settings.command.targetUncertainty));
+        final WorkQueue workQueue;
+        if (opCount < 0)
+            workQueue = new ContinuousWorkQueue(50);
+        else
+            workQueue = FixedWorkQueue.build(opCount);
+
+        RateLimiter rateLimiter = null;
+        // TODO : move this to a new queue wrapper that gates progress based on a poisson (or configurable) distribution
+        if (settings.rate.opRateTargetPerSecond > 0)
+            rateLimiter = RateLimiter.create(settings.rate.opRateTargetPerSecond);
+
+        final StressMetrics metrics = new StressMetrics(output, settings.log.intervalMillis);
+
+        final CountDownLatch done = new CountDownLatch(threadCount);
+        final Consumer[] consumers = new Consumer[threadCount];
+        for (int i = 0; i < threadCount; i++)
+            consumers[i] = new Consumer(type, done, workQueue, metrics, rateLimiter);
 
-                oldTotal = total;
-                oldKeyCount = keyCount;
+        // starting worker threadCount
+        for (int i = 0; i < threadCount; i++)
+            consumers[i].start();
 
-                total = client.operations.get();
-                keyCount = client.keys.get();
-                latency = client.latency.getSnapshot();
+        metrics.start();
 
-                int opDelta = total - oldTotal;
-                int keyDelta = keyCount - oldKeyCount;
+        if (opCount <= 0)
+        {
+            try
+            {
+                metrics.waitUntilConverges(settings.command.targetUncertainty,
+                        settings.command.minimumUncertaintyMeasurements,
+                        settings.command.maximumUncertaintyMeasurements);
+            } catch (InterruptedException e) { }
+            workQueue.stop();
+        }
 
-                long currentTimeInSeconds = TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - testStartTime);
+        try
+        {
+            done.await();
+            metrics.stop();
+        } catch (InterruptedException e) {}
 
-                output.println(String.format("%d,%d,%d,%.1f,%.1f,%.1f,%d",
-                                             total,
-                                             opDelta / interval,
-                                             keyDelta / interval,
-                                             latency.getMedian(), latency.get95thPercentile(), latency.get999thPercentile(),
-                                             currentTimeInSeconds));
+        if (metrics.wasCancelled())
+            return null;
 
-                if (client.outputStatistics()) {
-                    stats.addIntervalStats(total, 
-                                           opDelta / interval, 
-                                           keyDelta / interval, 
-                                           latency, 
-                                           currentTimeInSeconds);
-                        }
-            }
-        }
+        metrics.summarise();
 
-        // if any consumer failed, set the return code to failure.
-        returnCode = SUCCESS;
-        if (producer.isAlive())
-        {
-            producer.interrupt(); // if producer is still alive it means that we had errors in the consumers
-            returnCode = FAILURE;
-        }
+        boolean success = true;
         for (Consumer consumer : consumers)
-            if (consumer.getReturnCode() == FAILURE)
-                returnCode = FAILURE;
-
-        if (returnCode == SUCCESS) {            
-            if (client.outputStatistics())
-                stats.printStats();
-            // marking an end of the output to the client
-            output.println("END");            
-        } else {
-            output.println("FAILURE");
-        }
+            success &= consumer.success;
 
-    }
+        if (!success)
+            return null;
 
-    public int getReturnCode()
-    {
-        return returnCode;
+        return metrics;
     }
 
-    /**
-     * Produces exactly N items (awaits each to be consumed)
-     */
-    private class Producer extends Thread
+    private class Consumer extends Thread
     {
-        private volatile boolean stop = false;
+
+        private final Operation.State state;
+        private final RateLimiter rateLimiter;
+        private volatile boolean success = true;
+        private final WorkQueue workQueue;
+        private final CountDownLatch done;
+
+        public Consumer(Command type, CountDownLatch done, WorkQueue workQueue, StressMetrics metrics, RateLimiter rateLimiter)
+        {
+            this.done = done;
+            this.rateLimiter = rateLimiter;
+            this.workQueue = workQueue;
+            this.state = new Operation.State(type, settings, metrics);
+        }
 
         public void run()
         {
-            for (int i = 0; i < client.getNumKeys(); i++)
+
+            try
             {
-                if (stop)
-                    break;
 
-                try
+                SimpleClient sclient = null;
+                ThriftClient tclient = null;
+                JavaDriverClient jclient = null;
+
+                switch (settings.mode.api)
                 {
-                    operations.put(createOperation(i % client.getNumDifferentKeys()));
+                    case JAVA_DRIVER_NATIVE:
+                        jclient = settings.getJavaDriverClient();
+                        break;
+                    case SIMPLE_NATIVE:
+                        sclient = settings.getSimpleNativeClient();
+                        break;
+                    case THRIFT:
+                        tclient = settings.getThriftClient();
+                        break;
+                    case THRIFT_SMART:
+                        tclient = settings.getSmartThriftClient();
+                        break;
+                    default:
+                        throw new IllegalStateException();
                 }
-                catch (InterruptedException e)
+
+                Work work;
+                while ( null != (work = workQueue.poll()) )
                 {
-                    if (e.getMessage() != null)
-                        System.err.println("Producer error - " + e.getMessage());
-                    return;
+
+                    if (rateLimiter != null)
+                        rateLimiter.acquire(work.count);
+
+                    for (int i = 0 ; i < work.count ; i++)
+                    {
+                        try
+                        {
+                            Operation op = createOperation(state, i + work.offset);
+                            switch (settings.mode.api)
+                            {
+                                case JAVA_DRIVER_NATIVE:
+                                    op.run(jclient);
+                                    break;
+                                case SIMPLE_NATIVE:
+                                    op.run(sclient);
+                                    break;
+                                default:
+                                    op.run(tclient);
+                            }
+                        } catch (Exception e)
+                        {
+                            if (output == null)
+                            {
+                                System.err.println(e.getMessage());
+                                success = false;
+                                System.exit(-1);
+                            }
+
+                            e.printStackTrace(output);
+                            success = false;
+                            workQueue.stop();
+                            state.metrics.cancel();
+                            return;
+                        }
+                    }
                 }
+
             }
+            finally
+            {
+                done.countDown();
+                state.timer.close();
+            }
+
         }
 
-        public void stopProducer()
+    }
+
+    private interface WorkQueue
+    {
+        // null indicates consumer should terminate
+        Work poll();
+
+        // signal all consumers to terminate
+        void stop();
+    }
+
+    private static final class Work
+    {
+        // index of operations
+        final long offset;
+
+        // how many operations to perform
+        final int count;
+
+        public Work(long offset, int count)
         {
-            stop = true;
+            this.offset = offset;
+            this.count = count;
         }
     }
 
-    /**
-     * Each consumes exactly N items from queue
-     */
-    private class Consumer extends Thread
+    private static final class FixedWorkQueue implements WorkQueue
     {
-        private final int items;
-        private final RateLimiter rateLimiter;
-        private volatile boolean stop = false;
-        private volatile int returnCode = StressAction.SUCCESS;
 
-        public Consumer(int toConsume, RateLimiter rateLimiter)
+        final ArrayBlockingQueue<Work> work;
+        volatile boolean stop = false;
+
+        public FixedWorkQueue(ArrayBlockingQueue<Work> work)
         {
-            items = toConsume;
-            this.rateLimiter = rateLimiter;
+            this.work = work;
         }
 
-        public void run()
+        @Override
+        public Work poll()
+        {
+            if (stop)
+                return null;
+            return work.poll();
+        }
+
+        @Override
+        public void stop()
+        {
+            stop = true;
+        }
+
+        static FixedWorkQueue build(long operations)
         {
-            if (client.use_native_protocol)
+            // target splitting into around 50-500k items, with a minimum size of 20
+            if (operations > Integer.MAX_VALUE * (1L << 19))
+                throw new IllegalStateException("Cannot currently support more than approx 2^50 operations for one stress run. This is a LOT.");
+            int batchSize = (int) (operations / (1 << 19));
+            if (batchSize < 20)
+                batchSize = 20;
+            ArrayBlockingQueue<Work> work = new ArrayBlockingQueue<Work>(
+                    (int) ((operations / batchSize)
+                  + (operations % batchSize == 0 ? 0 : 1))
+            );
+            long offset = 0;
+            while (offset < operations)
             {
-                SimpleClient connection = client.getNativeClient();
+                work.add(new Work(offset, (int) Math.min(batchSize, operations - offset)));
+                offset += batchSize;
+            }
+            return new FixedWorkQueue(work);
+        }
 
-                for (int i = 0; i < items; i++)
-                {
-                    if (stop)
-                        break;
+    }
 
-                    try
-                    {
-                        rateLimiter.acquire();
-                        operations.take().run(connection); // running job
-                    }
-                    catch (Exception e)
-                    {
-                        if (output == null)
-                        {
-                            System.err.println(e.getMessage());
-                            returnCode = StressAction.FAILURE;
-                            System.exit(-1);
-                        }
+    private static final class ContinuousWorkQueue implements WorkQueue
+    {
 
-                        output.println(e.getMessage());
-                        returnCode = StressAction.FAILURE;
-                        break;
-                    }
-                }
-            }
-            else
-            {
-                CassandraClient connection = client.getClient();
+        final AtomicLong offset = new AtomicLong();
+        final int batchSize;
+        volatile boolean stop = false;
 
-                for (int i = 0; i < items; i++)
-                {
-                    if (stop)
-                        break;
+        private ContinuousWorkQueue(int batchSize)
+        {
+            this.batchSize = batchSize;
+        }
 
-                    try
-                    {
-                        rateLimiter.acquire();
-                        operations.take().run(connection); // running job
-                    }
-                    catch (Exception e)
-                    {
-                        if (output == null)
-                        {
-                            System.err.println(e.getMessage());
-                            returnCode = StressAction.FAILURE;
-                            System.exit(-1);
-                        }
+        @Override
+        public Work poll()
+        {
+            if (stop)
+                return null;
+            return new Work(nextOffset(), batchSize);
+        }
 
-                        output.println(e.getMessage());
-                        returnCode = StressAction.FAILURE;
-                        break;
-                    }
-                }
+        private long nextOffset()
+        {
+            final int inc = batchSize;
+            while (true)
+            {
+                final long cur = offset.get();
+                if (offset.compareAndSet(cur, cur + inc))
+                    return cur;
             }
         }
 
-        public void stopConsume()
+        @Override
+        public void stop()
         {
             stop = true;
         }
 
-        public int getReturnCode()
-        {
-            return returnCode;
-        }
     }
 
-    private Operation createOperation(int index)
+    private Operation createOperation(Operation.State state, long index)
+    {
+        return createOperation(state.type, state, index);
+    }
+    private Operation createOperation(Command type, Operation.State state, long index)
     {
-        switch (client.getOperation())
+        switch (type)
         {
             case READ:
-                return client.isCQL() ? new CqlReader(client, index) : new Reader(client, index);
+                switch(state.settings.mode.style)
+                {
+                    case THRIFT:
+                        return new ThriftReader(state, index);
+                    case CQL:
+                    case CQL_PREPARED:
+                        return new CqlReader(state, index);
+                    default:
+                        throw new UnsupportedOperationException();
+                }
+
+
+            case COUNTERREAD:
+                switch(state.settings.mode.style)
+                {
+                    case THRIFT:
+                        return new ThriftCounterGetter(state, index);
+                    case CQL:
+                    case CQL_PREPARED:
+                        return new CqlCounterGetter(state, index);
+                    default:
+                        throw new UnsupportedOperationException();
+                }
+
+            case WRITE:
+                switch(state.settings.mode.style)
+                {
+                    case THRIFT:
+                        return new ThriftInserter(state, index);
+                    case CQL:
+                    case CQL_PREPARED:
+                        return new CqlInserter(state, index);
+                    default:
+                        throw new UnsupportedOperationException();
+                }
 
-            case COUNTER_GET:
-                return client.isCQL() ? new CqlCounterGetter(client, index) : new CounterGetter(client, index);
+            case COUNTERWRITE:
+                switch(state.settings.mode.style)
+                {
+                    case THRIFT:
+                        return new ThriftCounterAdder(state, index);
+                    case CQL:
+                    case CQL_PREPARED:
+                        return new CqlCounterAdder(state, index);
+                    default:
+                        throw new UnsupportedOperationException();
+                }
 
-            case INSERT:
-                return client.isCQL() ? new CqlInserter(client, index) : new Inserter(client, index);
+            case RANGESLICE:
+                switch(state.settings.mode.style)
+                {
+                    case THRIFT:
+                        return new ThriftRangeSlicer(state, index);
+                    case CQL:
+                    case CQL_PREPARED:
+                        return new CqlRangeSlicer(state, index);
+                    default:
+                        throw new UnsupportedOperationException();
+                }
 
-            case COUNTER_ADD:
-                return client.isCQL() ? new CqlCounterAdder(client, index) : new CounterAdder(client, index);
+            case IRANGESLICE:
+                switch(state.settings.mode.style)
+                {
+                    case THRIFT:
+                        return new ThriftIndexedRangeSlicer(state, index);
+                    case CQL:
+                    case CQL_PREPARED:
+                        return new CqlIndexedRangeSlicer(state, index);
+                    default:
+                        throw new UnsupportedOperationException();
+                }
 
-            case RANGE_SLICE:
-                return client.isCQL() ? new CqlRangeSlicer(client, index) : new RangeSlicer(client, index);
+            case READMULTI:
+                switch(state.settings.mode.style)
+                {
+                    case THRIFT:
+                        return new ThriftMultiGetter(state, index);
+                    case CQL:
+                    case CQL_PREPARED:
+                        return new CqlMultiGetter(state, index);
+                    default:
+                        throw new UnsupportedOperationException();
+                }
 
-            case INDEXED_RANGE_SLICE:
-                return client.isCQL() ? new CqlIndexedRangeSlicer(client, index) : new IndexedRangeSlicer(client, index);
+            case MIXED:
+                return createOperation(state.readWriteSelector.next(), state, index);
 
-            case MULTI_GET:
-                return client.isCQL() ? new CqlMultiGetter(client, index) : new MultiGetter(client, index);
         }
 
         throw new UnsupportedOperationException();
     }
 
-    public void stopAction()
-    {
-        stop = true;
-    }
 }


[3/6] Improve Stress Tool patch by Benedict; reviewed by Pavel Yaskevich for CASSANDRA-6199

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/server/StressThread.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/server/StressThread.java b/tools/stress/src/org/apache/cassandra/stress/server/StressThread.java
deleted file mode 100644
index 158a09f..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/server/StressThread.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * 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.server;
-
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.StressAction;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.PrintStream;
-import java.net.Socket;
-
-public class StressThread extends Thread
-{
-    private final Socket socket;
-
-    public StressThread(Socket client)
-    {
-        this.socket = client;
-    }
-
-    public void run()
-    {
-        try
-        {
-            ObjectInputStream in = new ObjectInputStream(socket.getInputStream());
-            PrintStream out = new PrintStream(socket.getOutputStream());
-
-            StressAction action = new StressAction((Session) in.readObject(), out);
-            action.start();
-
-            while (action.isAlive())
-            {
-                try
-                {
-                    if (in.readInt() == 1)
-                    {
-                        action.stopAction();
-                        break;
-                    }
-                }
-                catch (Exception e)
-                {
-                    // continue without problem
-                }
-            }
-
-            out.close();
-            in.close();
-            socket.close();
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e.getMessage(), e);
-        }
-        catch (Exception e)
-        {
-            e.printStackTrace();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java b/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java
new file mode 100644
index 0000000..76c7509
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java
@@ -0,0 +1,58 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public enum CliOption
+{
+    KEY("Key details such as size in bytes and value distribution", SettingsKey.helpPrinter()),
+    COL("Column details such as size and count distribution, data generator, names, comparator and if super columns should be used", SettingsColumn.helpPrinter()),
+    RATE("Thread count, rate limit or automatic mode (default is auto)", SettingsRate.helpPrinter()),
+    MODE("Thrift or CQL with options", SettingsMode.helpPrinter()),
+    SCHEMA("Replication settings, compression, compaction, etc.", SettingsSchema.helpPrinter()),
+    NODE("Nodes to connect to", SettingsNode.helpPrinter()),
+    LOG("Where to log progress to, and the interval at which to do it", SettingsLog.helpPrinter()),
+    TRANSPORT("Custom transport factories", SettingsTransport.helpPrinter()),
+    PORT("The port to connect to cassandra nodes on", SettingsPort.helpPrinter()),
+    SENDTO("-send-to", "Specify a stress server to send this command to", SettingsMisc.sendToDaemonHelpPrinter())
+    ;
+
+    private static final Map<String, CliOption> LOOKUP;
+    static
+    {
+        final Map<String, CliOption> lookup = new HashMap<>();
+        for (CliOption cmd : values())
+        {
+            lookup.put("-" + cmd.toString().toLowerCase(), cmd);
+            if (cmd.extraName != null)
+                lookup.put(cmd.extraName, cmd);
+        }
+        LOOKUP = lookup;
+    }
+
+    public static CliOption get(String command)
+    {
+        return LOOKUP.get(command.toLowerCase());
+    }
+
+    public final String extraName;
+    public final String description;
+    private final Runnable helpPrinter;
+
+    private CliOption(String description, Runnable helpPrinter)
+    {
+        this(null, description, helpPrinter);
+    }
+    private CliOption(String extraName, String description, Runnable helpPrinter)
+    {
+        this.extraName = extraName;
+        this.description = description;
+        this.helpPrinter = helpPrinter;
+    }
+
+    public void printHelp()
+    {
+        helpPrinter.run();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/Command.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/Command.java b/tools/stress/src/org/apache/cassandra/stress/settings/Command.java
new file mode 100644
index 0000000..4bd843e
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/Command.java
@@ -0,0 +1,101 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public enum Command
+{
+
+    READ(false,
+            SettingsCommand.helpPrinter("read"),
+            "Multiple concurrent reads - the cluster must first be populated by a write test",
+            CommandCategory.BASIC
+    ),
+    WRITE(true,
+            SettingsCommand.helpPrinter("write"),
+            "insert",
+            "Multiple concurrent writes against the cluster",
+            CommandCategory.BASIC
+    ),
+    MIXED(true,
+            SettingsCommandMixed.helpPrinter(),
+            "Interleaving of any basic commands, with configurable ratio and distribution - the cluster must first be populated by a write test",
+            CommandCategory.MIXED
+    ),
+    RANGESLICE(false,
+            SettingsCommandMulti.helpPrinter("range_slice"),
+            "Range slice queries - the cluster must first be populated by a write test",
+            CommandCategory.MULTI
+    ),
+    IRANGESLICE(false,
+            SettingsCommandMulti.helpPrinter("indexed_range_slice"),
+            "Range slice queries through a secondary index. The cluster must first be populated by a write test, with indexing enabled.",
+            CommandCategory.MULTI
+    ),
+    READMULTI(false,
+            SettingsCommandMulti.helpPrinter("readmulti"),
+            "multi_read",
+            "Multiple concurrent reads fetching multiple rows at once. The cluster must first be populated by a write test.",
+            CommandCategory.MULTI
+    ),
+    COUNTERWRITE(true,
+            SettingsCommand.helpPrinter("counteradd"),
+            "counter_add",
+            "Multiple concurrent updates of counters.",
+            CommandCategory.BASIC
+    ),
+    COUNTERREAD(false,
+            SettingsCommand.helpPrinter("counterread"),
+            "counter_get",
+            "Multiple concurrent reads of counters. The cluster must first be populated by a counterwrite test.",
+            CommandCategory.BASIC
+    ),
+
+    HELP(false, SettingsMisc.helpHelpPrinter(), "-?", "Print help for a command or option", null),
+    PRINT(false, SettingsMisc.printHelpPrinter(), "Inspect the output of a distribution definition", null),
+    LEGACY(false, Legacy.helpPrinter(), "Legacy support mode", null)
+
+    ;
+
+    private static final Map<String, Command> LOOKUP;
+    static
+    {
+        final Map<String, Command> lookup = new HashMap<>();
+        for (Command cmd : values())
+        {
+            lookup.put(cmd.toString().toLowerCase(), cmd);
+            if (cmd.extraName != null)
+                lookup.put(cmd.extraName, cmd);
+        }
+        LOOKUP = lookup;
+    }
+
+    public static Command get(String command)
+    {
+        return LOOKUP.get(command.toLowerCase());
+    }
+
+    public final boolean updates;
+    public final CommandCategory category;
+    public final String extraName;
+    public final String description;
+    public final Runnable helpPrinter;
+
+    Command(boolean updates, Runnable helpPrinter, String description, CommandCategory category)
+    {
+        this(updates, helpPrinter, null, description, category);
+    }
+    Command(boolean updates, Runnable helpPrinter, String extra, String description, CommandCategory category)
+    {
+        this.updates = updates;
+        this.category = category;
+        this.helpPrinter = helpPrinter;
+        this.extraName = extra;
+        this.description = description;
+    }
+    public void printHelp()
+    {
+        helpPrinter.run();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/CommandCategory.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/CommandCategory.java b/tools/stress/src/org/apache/cassandra/stress/settings/CommandCategory.java
new file mode 100644
index 0000000..87a13f7
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/CommandCategory.java
@@ -0,0 +1,8 @@
+package org.apache.cassandra.stress.settings;
+
+public enum CommandCategory
+{
+    BASIC,
+    MULTI,
+    MIXED
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/ConnectionAPI.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/ConnectionAPI.java b/tools/stress/src/org/apache/cassandra/stress/settings/ConnectionAPI.java
new file mode 100644
index 0000000..c647f66
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/ConnectionAPI.java
@@ -0,0 +1,7 @@
+package org.apache.cassandra.stress.settings;
+
+public enum ConnectionAPI
+{
+    THRIFT, THRIFT_SMART, SIMPLE_NATIVE, JAVA_DRIVER_NATIVE
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/ConnectionStyle.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/ConnectionStyle.java b/tools/stress/src/org/apache/cassandra/stress/settings/ConnectionStyle.java
new file mode 100644
index 0000000..6e77f4a
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/ConnectionStyle.java
@@ -0,0 +1,9 @@
+package org.apache.cassandra.stress.settings;
+
+public enum ConnectionStyle
+{
+    CQL,
+    CQL_PREPARED,
+    THRIFT
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java b/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
new file mode 100644
index 0000000..853e399
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
@@ -0,0 +1,48 @@
+package org.apache.cassandra.stress.settings;
+
+public enum CqlVersion
+{
+
+    NOCQL(null),
+    CQL2("2.0.0"),
+    CQL3("3.0.0");
+
+    public final String connectVersion;
+
+    private CqlVersion(String connectVersion)
+    {
+        this.connectVersion = connectVersion;
+    }
+
+    static CqlVersion get(String version)
+    {
+        if (version == null)
+            return NOCQL;
+        switch(version.charAt(0))
+        {
+            case '2':
+                return CQL2;
+            case '3':
+                return CQL3;
+            default:
+                throw new IllegalStateException();
+        }
+    }
+
+    public boolean isCql()
+    {
+        return this != NOCQL;
+    }
+
+    public boolean isCql2()
+    {
+        return this == CQL2;
+    }
+
+    public boolean isCql3()
+    {
+        return this == CQL3;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/GroupedOptions.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/GroupedOptions.java b/tools/stress/src/org/apache/cassandra/stress/settings/GroupedOptions.java
new file mode 100644
index 0000000..fe965c9
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/GroupedOptions.java
@@ -0,0 +1,104 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.PrintStream;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public abstract class GroupedOptions
+{
+
+    int accepted = 0;
+
+    public boolean accept(String param)
+    {
+        for (Option option : options())
+        {
+            if (option.accept(param))
+            {
+                accepted++;
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public boolean happy()
+    {
+        for (Option option : options())
+            if (!option.happy())
+                return false;
+        return true;
+    }
+
+    public abstract List<? extends Option> options();
+
+    // hands the parameters to each of the option groups, and returns the first provided
+    // option group that is happy() after this is done, that also accepted all the parameters
+    public static <G extends GroupedOptions> G select(String[] params, G... groupings)
+    {
+        for (String param : params)
+        {
+            boolean accepted = false;
+            for (GroupedOptions grouping : groupings)
+                accepted |= grouping.accept(param);
+            if (!accepted)
+                throw new IllegalArgumentException("Invalid parameter " + param);
+        }
+        for (G grouping : groupings)
+            if (grouping.happy() && grouping.accepted == params.length)
+                return grouping;
+        return null;
+    }
+
+    // pretty prints all of the option groupings
+    public static void printOptions(PrintStream out, String command, GroupedOptions... groupings)
+    {
+        out.println();
+        boolean firstRow = true;
+        for (GroupedOptions grouping : groupings)
+        {
+            if (!firstRow)
+            {
+                out.println(" OR ");
+            }
+            firstRow = false;
+
+            StringBuilder sb = new StringBuilder("Usage: " + command);
+            for (Option option : grouping.options())
+            {
+                sb.append(" ");
+                sb.append(option.shortDisplay());
+            }
+            out.println(sb.toString());
+        }
+        out.println();
+        final Set<Option> printed = new HashSet<>();
+        for (GroupedOptions grouping : groupings)
+        {
+            for (Option option : grouping.options())
+            {
+                if (printed.add(option))
+                {
+                    if (option.longDisplay() != null)
+                    {
+                        out.println("  " + option.longDisplay());
+                        for (String row : option.multiLineDisplay())
+                            out.println("      " + row);
+                    }
+                }
+            }
+        }
+    }
+
+    public static String formatLong(String longDisplay, String description)
+    {
+        return String.format("%-40s %s", longDisplay, description);
+    }
+
+    public static String formatMultiLine(String longDisplay, String description)
+    {
+        return String.format("%-36s %s", longDisplay, description);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java b/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
new file mode 100644
index 0000000..6242425
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
@@ -0,0 +1,369 @@
+/**
+ * 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.settings;
+
+import java.io.Serializable;
+import java.util.*;
+
+import org.apache.commons.cli.*;
+import org.apache.commons.cli.Option;
+
+public class Legacy implements Serializable
+{
+
+    // command line options
+    public static final Options availableOptions = new Options();
+
+    private static final String SSL_TRUSTSTORE = "truststore";
+    private static final String SSL_TRUSTSTORE_PW = "truststore-password";
+    private static final String SSL_PROTOCOL = "ssl-protocol";
+    private static final String SSL_ALGORITHM = "ssl-alg";
+    private static final String SSL_STORE_TYPE = "store-type";
+    private static final String SSL_CIPHER_SUITES = "ssl-ciphers";
+
+    static
+    {
+        availableOptions.addOption("h",  "help",                 false,  "Show this help message and exit");
+        availableOptions.addOption("n",  "num-keys",             true,   "Number of keys, default:1000000");
+        availableOptions.addOption("F",  "num-different-keys",   true,   "Number of different keys (if < NUM-KEYS, the same key will re-used multiple times), default:NUM-KEYS");
+        availableOptions.addOption("t",  "threadCount",              true,   "Number of threadCount to use, default:50");
+        availableOptions.addOption("c",  "columns",              true,   "Number of columns per key, default:5");
+        availableOptions.addOption("S",  "column-size",          true,   "Size of column values in bytes, default:34");
+        availableOptions.addOption("C",  "unique columns",       true,   "Max number of unique columns per key, default:50");
+        availableOptions.addOption("RC", "unique rows",          true,   "Max number of unique rows, default:50");
+        availableOptions.addOption("d",  "nodes",                true,   "Host nodes (comma separated), default:locahost");
+        availableOptions.addOption("D",  "nodesfile",            true,   "File containing host nodes (one per line)");
+        availableOptions.addOption("s",  "stdev",                true,   "Standard Deviation for gaussian read key generation, default:0.1");
+        availableOptions.addOption("r",  "random",               false,  "Use random key generator for read key generation (STDEV will have no effect), default:false");
+        availableOptions.addOption("f",  "file",                 true,   "Write output to given file");
+        availableOptions.addOption("p",  "port",                 true,   "Thrift port, default:9160");
+        availableOptions.addOption("o",  "operation",            true,   "Operation to perform (WRITE, READ, READWRITE, RANGE_SLICE, INDEXED_RANGE_SLICE, MULTI_GET, COUNTERWRITE, COUNTER_GET), default:WRITE");
+        availableOptions.addOption("u",  "supercolumns",         true,   "Number of super columns per key, default:1");
+        availableOptions.addOption("y",  "family-type",          true,   "Column Family Type (Super, Standard), default:Standard");
+        availableOptions.addOption("K",  "keep-trying",          true,   "Retry on-going operation N times (in case of failure). positive integer, default:10");
+        availableOptions.addOption("k",  "keep-going",           false,  "Ignore errors inserting or reading (when set, --keep-trying has no effect), default:false");
+        availableOptions.addOption("i",  "progress-interval",    true,   "Progress Report Interval (seconds), default:10");
+        availableOptions.addOption("g",  "keys-per-call",        true,   "Number of keys to get_range_slices or multiget per call, default:1000");
+        availableOptions.addOption("l",  "replication-factor",   true,   "Replication Factor to use when creating needed column families, default:1");
+        availableOptions.addOption("L",  "enable-cql",           false,  "Perform queries using CQL2 (Cassandra Query Language v 2.0.0)");
+        availableOptions.addOption("L3", "enable-cql3",          false,  "Perform queries using CQL3 (Cassandra Query Language v 3.0.0)");
+        availableOptions.addOption("b",  "enable-native-protocol",  false,  "Use the binary native protocol (only work along with -L3)");
+        availableOptions.addOption("P",  "use-prepared-statements", false, "Perform queries using prepared statements (only applicable to CQL).");
+        availableOptions.addOption("e",  "consistency-level",    true,   "Consistency Level to use (ONE, QUORUM, LOCAL_QUORUM, EACH_QUORUM, ALL, ANY), default:ONE");
+        availableOptions.addOption("x",  "create-index",         true,   "Type of index to create on needed column families (KEYS)");
+        availableOptions.addOption("R",  "replication-strategy", true,   "Replication strategy to use (only on insert if keyspace does not exist), default:org.apache.cassandra.locator.SimpleStrategy");
+        availableOptions.addOption("O",  "strategy-properties",  true,   "Replication strategy properties in the following format <dc_name>:<num>,<dc_name>:<num>,...");
+        availableOptions.addOption("W",  "no-replicate-on-write",false,  "Set replicate_on_write to false for counters. Only counter add with CL=ONE will work");
+        availableOptions.addOption("V",  "average-size-values",  false,  "Generate column values of average rather than specific size");
+        availableOptions.addOption("T",  "send-to",              true,   "Send this as a request to the stress daemon at specified address.");
+        availableOptions.addOption("I",  "compression",          true,   "Specify the compression to use for sstable, default:no compression");
+        availableOptions.addOption("Q",  "query-names",          true,   "Comma-separated list of column names to retrieve from each row.");
+        availableOptions.addOption("Z",  "compaction-strategy",  true,   "CompactionStrategy to use.");
+        availableOptions.addOption("U",  "comparator",           true,   "Column Comparator to use. Currently supported types are: TimeUUIDType, AsciiType, UTF8Type.");
+        availableOptions.addOption("tf", "transport-factory",    true,   "Fully-qualified TTransportFactory class name for creating a connection. Note: For Thrift over SSL, use org.apache.cassandra.stress.SSLTransportFactory.");
+        availableOptions.addOption("ns", "no-statistics",        false,  "Turn off the aggegate statistics that is normally output after completion.");
+        availableOptions.addOption("ts", SSL_TRUSTSTORE,         true, "SSL: full path to truststore");
+        availableOptions.addOption("tspw", SSL_TRUSTSTORE_PW,    true, "SSL: full path to truststore");
+        availableOptions.addOption("prtcl", SSL_PROTOCOL,        true, "SSL: connections protocol to use (default: TLS)");
+        availableOptions.addOption("alg", SSL_ALGORITHM,         true, "SSL: algorithm (default: SunX509)");
+        availableOptions.addOption("st", SSL_STORE_TYPE,         true, "SSL: type of store");
+        availableOptions.addOption("ciphers", SSL_CIPHER_SUITES, true, "SSL: comma-separated list of encryption suites to use");
+        availableOptions.addOption("th",  "throttle",            true,   "Throttle the total number of operations per second to a maximum amount.");
+    }
+
+    public static StressSettings build(String[] arguments)
+    {
+        CommandLineParser parser = new PosixParser();
+
+        final Converter r = new Converter();
+        try
+        {
+            CommandLine cmd = parser.parse(availableOptions, arguments);
+
+            if (cmd.getArgs().length > 0)
+            {
+                System.err.println("Application does not allow arbitrary arguments: " + Arrays.asList(cmd.getArgList()));
+                System.exit(1);
+            }
+
+            if (cmd.hasOption("h"))
+                printHelpMessage();
+
+            if (cmd.hasOption("C"))
+                System.out.println("Ignoring deprecated option -C");
+
+            if (cmd.hasOption("o"))
+                r.setCommand(cmd.getOptionValue("o").toLowerCase());
+            else
+                r.setCommand("insert");
+
+            if (cmd.hasOption("K"))
+                r.add("command", "tries=" + cmd.getOptionValue("K"));
+
+            if (cmd.hasOption("k"))
+            {
+                if (!cmd.hasOption("K"))
+                    r.add("command", "retry=1");
+                r.add("command", "ignore_errors");
+            }
+
+            if (cmd.hasOption("g"))
+                r.add("command", "at-once=" + cmd.getOptionValue("g"));
+
+            if (cmd.hasOption("e"))
+                r.add("command", "cl=" + cmd.getOptionValue("e"));
+
+            String numKeys;
+            if (cmd.hasOption("n"))
+                numKeys = cmd.getOptionValue("n");
+            else
+                numKeys = "1000000";
+            r.add("command", "n=" + numKeys);
+
+            String uniqueKeys;
+            if (cmd.hasOption("F"))
+                uniqueKeys = cmd.getOptionValue("F");
+            else
+                uniqueKeys = numKeys;
+
+            if (r.opts.containsKey("write") || r.opts.containsKey("counterwrite"))
+            {
+                if (!uniqueKeys.equals(numKeys))
+                    r.add("-key", "populate=1.." + uniqueKeys);
+            }
+            else if (cmd.hasOption("r"))
+            {
+                r.add("-key", "dist=uniform(1.." + uniqueKeys + ")");
+            }
+            else
+            {
+                if (!cmd.hasOption("s"))
+                    r.add("-key", "dist=gauss(1.." + uniqueKeys + ",5)");
+                else
+                    r.add("-key", String.format("dist=gauss(1..%s,%.2f)", uniqueKeys,
+                            0.5 / Float.parseFloat(cmd.getOptionValue("s"))));
+            }
+
+            String colCount;
+            if (cmd.hasOption("c"))
+                colCount = cmd.getOptionValue("c");
+            else
+                colCount = "5";
+
+            String colSize;
+            if (cmd.hasOption("S"))
+                colSize = cmd.getOptionValue("S");
+            else
+                colSize = "34";
+
+            r.add("-col", "n=fixed(" + colCount + ")");
+            if (cmd.hasOption("V"))
+            {
+                r.add("-col", "size=uniform(1.." + Integer.parseInt(colSize) * 2 + ")");
+                r.add("-col", "data=rand()");
+            }
+            else
+            {
+                r.add("-col", "size=fixed(" + colSize + ")");
+                r.add("-col", "data=repeat(1)");
+            }
+            if (cmd.hasOption("Q"))
+                r.add("-col", "names=" + cmd.getOptionValue("Q"));
+
+            if (cmd.hasOption("U"))
+                r.add("-col", "comparator=" + cmd.getOptionValue("U"));
+
+            if (cmd.hasOption("y") && cmd.getOptionValue("y").equals("Super"))
+                r.add("-col", "super=" + (cmd.hasOption("u") ? cmd.getOptionValue("u") : "1"));
+
+            if (cmd.hasOption("t"))
+                r.add("-rate", "threads=" + cmd.getOptionValue("t"));
+            else
+                r.add("-rate", "threads=50");
+
+            if (cmd.hasOption("th"))
+                r.add("-rate", "limit=" + cmd.getOptionValue("th") + "/s");
+
+            if (cmd.hasOption("f"))
+                r.add("-log", "file=" + cmd.getOptionValue("f"));
+
+            if (cmd.hasOption("p"))
+                r.add("-port", cmd.getOptionValue("p"));
+
+            if (cmd.hasOption("i"))
+                r.add("-log", "interval=" + cmd.getOptionValue("i"));
+            else
+                r.add("-log", "interval=10");
+
+            if (cmd.hasOption("x"))
+                r.add("-schema", "index=" + cmd.getOptionValue("x"));
+
+            if (cmd.hasOption("R") || cmd.hasOption("l") || cmd.hasOption("O"))
+            {
+                StringBuilder rep = new StringBuilder();
+                if (cmd.hasOption("R"))
+                    rep.append("strategy=" + cmd.getOptionValue("R"));
+                if (cmd.hasOption("l"))
+                {
+                    if (rep.length() > 0)
+                        rep.append(",");
+                    rep.append("factor=" + cmd.getOptionValue("l"));
+                }
+                if (cmd.hasOption("O"))
+                {
+                    if (rep.length() > 0)
+                        rep.append(",");
+                    rep.append(cmd.getOptionValue("O").replace(':','='));
+                }
+                r.add("-schema", "replication(" + rep + ")");
+            }
+
+            if (cmd.hasOption("L"))
+                r.add("-mode", cmd.hasOption("P") ? "prepared cql2" : "cql2");
+            else if (cmd.hasOption("L3"))
+                r.add("-mode", (cmd.hasOption("P") ? "prepared" : "") + (cmd.hasOption("b") ? "native" : "") +  "cql3");
+            else
+                r.add("-mode", "thrift");
+
+            if (cmd.hasOption("W"))
+                r.add("-schema", "no-replicate-on-write");
+
+            if (cmd.hasOption("I"))
+                r.add("-schema", "compression=" + cmd.getOptionValue("I"));
+
+            if (cmd.hasOption("d"))
+                r.add("-node", cmd.getOptionValue("d"));
+
+            if (cmd.hasOption("D"))
+                r.add("-node", "file=" + cmd.getOptionValue("D"));
+
+
+            if (cmd.hasOption("send-to"))
+                r.add("-send-to", cmd.getOptionValue("send-to"));
+
+            if (cmd.hasOption("Z"))
+                r.add("-schema", "compaction=" + cmd.getOptionValue("Z"));
+
+            if (cmd.hasOption("ns"))
+                r.add("-log", "no-summary");
+
+            if (cmd.hasOption("tf"))
+                r.add("-transport", "factory=" + cmd.getOptionValue("tf"));
+
+            // THESE DON'T SEEM TO AFFECT PROGRAM BEHAVIOUR
+//            if(cmd.hasOption(SSL_TRUSTSTORE))
+//                encOptions.truststore = cmd.getOptionValue(SSL_TRUSTSTORE);
+//
+//            if(cmd.hasOption(SSL_TRUSTSTORE_PW))
+//                encOptions.truststore_password = cmd.getOptionValue(SSL_TRUSTSTORE_PW);
+//
+//            if(cmd.hasOption(SSL_PROTOCOL))
+//                encOptions.protocol = cmd.getOptionValue(SSL_PROTOCOL);
+//
+//            if(cmd.hasOption(SSL_ALGORITHM))
+//                encOptions.algorithm = cmd.getOptionValue(SSL_ALGORITHM);
+//
+//            if(cmd.hasOption(SSL_STORE_TYPE))
+//                encOptions.store_type = cmd.getOptionValue(SSL_STORE_TYPE);
+//
+//            if(cmd.hasOption(SSL_CIPHER_SUITES))
+//                encOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(",");
+
+        }
+        catch (ParseException e)
+        {
+            printHelpMessage();
+            System.exit(1);
+        }
+
+        r.printNewCommand();
+        return r.get();
+    }
+
+    private static final class Converter
+    {
+        private Map<String, List<String>> opts = new LinkedHashMap<>();
+        List<String> command;
+        public void add(String option, String suboption)
+        {
+            if (option.equals("command"))
+            {
+                command.add(suboption);
+                return;
+            }
+            List<String> params = opts.get(option);
+            if (params == null)
+                opts.put(option, params = new ArrayList());
+            params.add(suboption);
+        }
+        StressSettings get(){
+            Map<String, String[]> clArgs = new HashMap<>();
+            for (Map.Entry<String, List<String>> e : opts.entrySet())
+                clArgs .put(e.getKey(), e.getValue().toArray(new String[0]));
+            return StressSettings.get(clArgs);
+        }
+        void setCommand(String command)
+        {
+            command = Command.get(command).toString().toLowerCase();
+            opts.put(command, this.command = new ArrayList<>());
+        }
+        void printNewCommand()
+        {
+            StringBuilder sb = new StringBuilder("stress");
+            for (Map.Entry<String, List<String>> e : opts.entrySet())
+            {
+                sb.append(" ");
+                sb.append(e.getKey());
+                for (String opt : e.getValue())
+                {
+                    sb.append(" ");
+                    sb.append(opt);
+                }
+            }
+            System.out.println("Running in legacy support mode. Translating command to: ");
+            System.out.println(sb.toString());
+        }
+    }
+
+    public static void printHelpMessage()
+    {
+        System.out.println("Usage: ./bin/cassandra-stress legacy [options]\n\nOptions:");
+        System.out.println("THIS IS A LEGACY SUPPORT MODE");
+
+        for(Object o : availableOptions.getOptions())
+        {
+            Option option = (Option) o;
+            String upperCaseName = option.getLongOpt().toUpperCase();
+            System.out.println(String.format("-%s%s, --%s%s%n\t\t%s%n", option.getOpt(), (option.hasArg()) ? " "+upperCaseName : "",
+                    option.getLongOpt(), (option.hasArg()) ? "="+upperCaseName : "", option.getDescription()));
+        }
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelpMessage();
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/Option.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/Option.java b/tools/stress/src/org/apache/cassandra/stress/settings/Option.java
new file mode 100644
index 0000000..dfd0857
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/Option.java
@@ -0,0 +1,24 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.List;
+
+abstract class Option
+{
+
+    abstract boolean accept(String param);
+    abstract boolean happy();
+    abstract String shortDisplay();
+    abstract String longDisplay();
+    abstract List<String> multiLineDisplay();
+
+    public int hashCode()
+    {
+        return getClass().hashCode();
+    }
+
+    public boolean equals(Object that)
+    {
+        return this.getClass() == that.getClass();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..7f4e5ea
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionDataGen.java
@@ -0,0 +1,177 @@
+package org.apache.cassandra.stress.settings;
+
+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;
+    }
+
+    @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/2e1e98ad/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
new file mode 100644
index 0000000..749c797
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java
@@ -0,0 +1,340 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.cassandra.stress.generatedata.*;
+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;
+
+/**
+ * For selecting a mathematical distribution
+ */
+class OptionDistribution 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 String spec;
+    private final String defaultSpec;
+
+    public OptionDistribution(String prefix, String defaultSpec)
+    {
+        this.prefix = prefix;
+        this.defaultSpec = defaultSpec;
+    }
+
+    @Override
+    public boolean accept(String param)
+    {
+        if (!param.toLowerCase().startsWith(prefix))
+            return false;
+        spec = param.substring(prefix.length());
+        return true;
+    }
+
+    private 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);
+        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));
+        while (m.find())
+            params.add(m.group());
+        return impl.getFactory(params);
+    }
+
+    public DistributionFactory get()
+    {
+        return spec != null ? get(spec) : get(defaultSpec);
+    }
+
+    @Override
+    public boolean happy()
+    {
+        return spec != null || defaultSpec != null;
+    }
+
+    public String longDisplay()
+    {
+        return shortDisplay() + ": Specify a mathematical distribution";
+    }
+
+    @Override
+    public List<String> multiLineDisplay()
+    {
+        return Arrays.asList(
+                GroupedOptions.formatMultiLine("EXP(min..max)", "An exponential distribution over the range [min..max]"),
+                GroupedOptions.formatMultiLine("EXTREME(min..max,shape)", "An extreme value (Weibull) distribution over the range [min..max]"),
+                GroupedOptions.formatMultiLine("GAUSSIAN(min..max,stdvrng)", "A gaussian/normal distribution, where mean=(min+max)/2, and stdev is (mean-min)/stdvrng"),
+                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"),
+                "Aliases: extr, gauss, normal, norm, weibull"
+        );
+    }
+
+    @Override
+    public String shortDisplay()
+    {
+        return prefix + "DIST(?)";
+    }
+
+    private static final Map<String, Impl> LOOKUP;
+    static
+    {
+        final Map<String, Impl> lookup = new HashMap<>();
+        lookup.put("exp", new ExponentialImpl());
+        lookup.put("extr", new ExtremeImpl());
+        lookup.put("extreme", lookup.get("extreme"));
+        lookup.put("weibull", lookup.get("weibull"));
+        lookup.put("gaussian", new GaussianImpl());
+        lookup.put("normal", lookup.get("gaussian"));
+        lookup.put("gauss", lookup.get("gaussian"));
+        lookup.put("norm", lookup.get("gaussian"));
+        lookup.put("uniform", new UniformImpl());
+        lookup.put("fixed", new FixedImpl());
+        LOOKUP = lookup;
+    }
+
+    // factory builders
+
+    private static interface Impl
+    {
+        public DistributionFactory getFactory(List<String> params);
+    }
+
+    private static final class GaussianImpl implements Impl
+    {
+
+        @Override
+        public DistributionFactory getFactory(List<String> params)
+        {
+            if (params.size() > 3 || params.size() < 1)
+                throw new IllegalArgumentException("Invalid parameter list for gaussian distribution: " + params);
+            try
+            {
+                String[] bounds = params.get(0).split("\\.\\.+");
+                final long min = Long.parseLong(bounds[0]);
+                final long max = Long.parseLong(bounds[1]);
+                final double mean, stdev;
+                if (params.size() == 3)
+                {
+                    mean = Double.parseDouble(params.get(1));
+                    stdev = Double.parseDouble(params.get(2));
+                }
+                else
+                {
+                    final double stdevsToEdge = params.size() == 1 ? 3d : Double.parseDouble(params.get(1));
+                    mean = (min + max) / 2d;
+                    stdev = ((max - min) / 2d) / stdevsToEdge;
+                }
+                return new GaussianFactory(min, max, mean, stdev);
+            } catch (Exception _)
+            {
+                throw new IllegalArgumentException("Invalid parameter list for uniform distribution: " + params);
+            }
+        }
+    }
+
+    private static final class ExponentialImpl implements Impl
+    {
+        @Override
+        public DistributionFactory getFactory(List<String> params)
+        {
+            if (params.size() != 1)
+                throw new IllegalArgumentException("Invalid parameter list for gaussian distribution: " + params);
+            try
+            {
+                String[] bounds = params.get(0).split("\\.\\.+");
+                final long min = Long.parseLong(bounds[0]);
+                final long max = Long.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
+                final double mean = (max - min) / findBounds.inverseCumulativeProbability(1d - Math.sqrt(1d/(max-min)));
+                return new ExpFactory(min, max, mean);
+            } catch (Exception _)
+            {
+                throw new IllegalArgumentException("Invalid parameter list for uniform distribution: " + params);
+            }
+        }
+    }
+
+    private static final class ExtremeImpl implements Impl
+    {
+        @Override
+        public DistributionFactory getFactory(List<String> params)
+        {
+            if (params.size() != 2)
+                throw new IllegalArgumentException("Invalid parameter list for extreme (Weibull) distribution: " + params);
+            try
+            {
+                String[] bounds = params.get(0).split("\\.\\.+");
+                final long min = Long.parseLong(bounds[0]);
+                final long max = Long.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,
+                // over entire range, but this results in overly skewed distribution, so take sqrt
+                final double scale = (max - min) / findBounds.inverseCumulativeProbability(1d - Math.sqrt(1d/(max-min)));
+                return new ExtremeFactory(min, max, shape, scale);
+            } catch (Exception _)
+            {
+                throw new IllegalArgumentException("Invalid parameter list for extreme (Weibull) distribution: " + params);
+            }
+        }
+    }
+
+    private static final class UniformImpl implements Impl
+    {
+
+        @Override
+        public DistributionFactory getFactory(List<String> params)
+        {
+            if (params.size() != 1)
+                throw new IllegalArgumentException("Invalid parameter list for uniform distribution: " + params);
+            try
+            {
+                String[] bounds = params.get(0).split("\\.\\.+");
+                final long min = Long.parseLong(bounds[0]);
+                final long max = Long.parseLong(bounds[1]);
+                return new UniformFactory(min, max);
+            } catch (Exception _)
+            {
+                throw new IllegalArgumentException("Invalid parameter list for uniform distribution: " + params);
+            }
+        }
+    }
+
+    private static final class FixedImpl implements Impl
+    {
+
+        @Override
+        public DistributionFactory getFactory(List<String> params)
+        {
+            if (params.size() != 1)
+                throw new IllegalArgumentException("Invalid parameter list for uniform distribution: " + params);
+            try
+            {
+                final long key = Long.parseLong(params.get(0));
+                return new FixedFactory(key);
+            } catch (Exception _)
+            {
+                throw new IllegalArgumentException("Invalid parameter list for uniform distribution: " + params);
+            }
+        }
+    }
+
+    // factories
+
+    private static final class ExpFactory implements DistributionFactory
+    {
+        final long min, max;
+        final double mean;
+        private ExpFactory(long min, long max, double mean)
+        {
+            this.min = min;
+            this.max = max;
+            this.mean = mean;
+        }
+
+        @Override
+        public Distribution get()
+        {
+            return new DistributionOffsetApache(new ExponentialDistribution(mean), min, max);
+        }
+    }
+
+    private static final class ExtremeFactory implements DistributionFactory
+    {
+        final long min, max;
+        final double shape, scale;
+        private ExtremeFactory(long min, long max, double shape, double scale)
+        {
+            this.min = min;
+            this.max = max;
+            this.shape = shape;
+            this.scale = scale;
+        }
+
+        @Override
+        public Distribution get()
+        {
+            return new DistributionOffsetApache(new WeibullDistribution(shape, scale), min, max);
+        }
+    }
+
+    private static final class GaussianFactory implements DistributionFactory
+    {
+        final long min, max;
+        final double mean, stdev;
+        private GaussianFactory(long min, long max, double mean, double stdev)
+        {
+            this.min = min;
+            this.max = max;
+            this.stdev = stdev;
+            this.mean = mean;
+        }
+
+        @Override
+        public Distribution get()
+        {
+            return new DistributionBoundApache(new NormalDistribution(mean, stdev), min, max);
+        }
+    }
+
+    private static final class UniformFactory implements DistributionFactory
+    {
+        final long min, max;
+        private UniformFactory(long min, long max)
+        {
+            this.min = min;
+            this.max = max;
+        }
+
+        @Override
+        public Distribution get()
+        {
+            return new DistributionBoundApache(new UniformRealDistribution(min, max), min, max);
+        }
+    }
+
+    private static final class FixedFactory implements DistributionFactory
+    {
+        final long key;
+        private FixedFactory(long key)
+        {
+            this.key = key;
+        }
+
+        @Override
+        public Distribution get()
+        {
+            return new DistributionFixed(key);
+        }
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return prefix.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return super.equals(that) && ((OptionDistribution) that).prefix.equals(this.prefix);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..9b92462
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionMulti.java
@@ -0,0 +1,107 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * For specifying multiple grouped sub-options in the form: group(arg1=,arg2,arg3) etc.
+ */
+abstract class OptionMulti extends Option
+{
+
+    private static final Pattern ARGS = Pattern.compile("([^,]+)", Pattern.CASE_INSENSITIVE);
+
+    private final class Delegate extends GroupedOptions
+    {
+        @Override
+        public List<? extends Option> options()
+        {
+            return OptionMulti.this.options();
+        }
+    }
+
+    protected abstract List<? extends Option> options();
+
+    private final String name;
+    private final Pattern pattern;
+    private final String description;
+    private final Delegate delegate = new Delegate();
+    public OptionMulti(String name, String description)
+    {
+        this.name = name;
+        pattern = Pattern.compile(name + "\\((.*)\\)", Pattern.CASE_INSENSITIVE);
+        this.description = description;
+    }
+
+    @Override
+    public boolean accept(String param)
+    {
+        Matcher m = pattern.matcher(param);
+        if (!m.matches())
+            return false;
+        m = ARGS.matcher(m.group(1));
+        int last = -1;
+        while (m.find())
+        {
+            if (m.start() != last + 1)
+                throw new IllegalArgumentException("Invalid " + name + " specification: " + param);
+            last = m.end();
+            if (!delegate.accept(m.group()))
+                throw new IllegalArgumentException("Invalid " + name + " specification: " + m.group());
+        }
+        return true;
+    }
+
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(name);
+        sb.append("(");
+        for (Option option : options())
+        {
+            sb.append(option);
+            sb.append(",");
+        }
+        sb.append(")");
+        return sb.toString();
+    }
+
+    @Override
+    public String shortDisplay()
+    {
+        return name + "(?)";
+    }
+
+    @Override
+    public String longDisplay()
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(name);
+        sb.append("(");
+        for (Option opt : options())
+        {
+            sb.append(opt.shortDisplay());
+        }
+        sb.append("): ");
+        sb.append(description);
+        return sb.toString();
+    }
+
+    @Override
+    public List<String> multiLineDisplay()
+    {
+        final List<String> r = new ArrayList<>();
+        for (Option option : options())
+            r.add(option.longDisplay());
+        return r;
+    }
+
+    @Override
+    boolean happy()
+    {
+        return delegate.happy();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/OptionReplication.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionReplication.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionReplication.java
new file mode 100644
index 0000000..b145de4
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionReplication.java
@@ -0,0 +1,114 @@
+package org.apache.cassandra.stress.settings;
+
+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.cassandra.locator.AbstractReplicationStrategy;
+
+/**
+ * For specifying replication options
+ */
+class OptionReplication extends Option
+{
+
+    private static final Pattern FULL = Pattern.compile("replication\\((.*)\\)", Pattern.CASE_INSENSITIVE);
+    private static final Pattern OPTION = Pattern.compile("([^,=]+)=([^,]+)", Pattern.CASE_INSENSITIVE);
+
+    private String strategy = "org.apache.cassandra.locator.SimpleStrategy";
+    private Map<String, String> options = new HashMap<>();
+
+    public String getStrategy()
+    {
+        return strategy;
+    }
+
+    public Map<String, String> getOptions()
+    {
+        if (!options.containsKey("replication_factor") && strategy.endsWith("SimpleStrategy"))
+            options.put("replication_factor", "1");
+        return options;
+    }
+
+
+    @Override
+    public boolean accept(String param)
+    {
+        Matcher m = FULL.matcher(param);
+        if (!m.matches())
+            return false;
+        String args = m.group(1);
+        m = OPTION.matcher(args);
+        int last = -1;
+        while (m.find())
+        {
+            if (m.start() != last + 1)
+                throw new IllegalArgumentException("Invalid replication specification: " + param);
+            last = m.end();
+            String key = m.group(1).toLowerCase();
+            sw: switch(key)
+            {
+                case "factor":
+                    try
+                    {
+                        Integer.parseInt(m.group(2));
+                    } catch (NumberFormatException e)
+                    {
+                        throw new IllegalArgumentException("Invalid replication factor: " + param);
+                    }
+                    options.put("replication_factor", m.group(2));
+                    break;
+                case "strategy":
+                    for (String name : new String[] { m.group(2), "org.apache.cassandra.locator." + m.group(2) })
+                    {
+                        try
+                        {
+                            Class<?> clazz = Class.forName(name);
+                            if (!AbstractReplicationStrategy.class.isAssignableFrom(clazz))
+                                throw new RuntimeException();
+                            strategy = name;
+                            break sw;
+                        } catch (Exception _)
+                        {
+                        }
+                    }
+                    throw new IllegalArgumentException("Invalid replication strategy: " + param);
+                default:
+
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public boolean happy()
+    {
+        return true;
+    }
+
+    @Override
+    public String shortDisplay()
+    {
+        return "replication(?)";
+    }
+
+    @Override
+    public String longDisplay()
+    {
+        return "replication(factor=?,strategy=?,<option1>=?,...)";
+    }
+
+    @Override
+    public List<String> multiLineDisplay()
+    {
+        return Arrays.asList(
+                GroupedOptions.formatMultiLine("factor=?","The replication factor to use (default 1)"),
+                GroupedOptions.formatMultiLine("strategy=?","The replication strategy to use (default SimpleStrategy)"),
+                GroupedOptions.formatMultiLine("option=?","Arbitrary replication strategy options")
+        );
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..01e75b5
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionSimple.java
@@ -0,0 +1,131 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Pattern;
+
+/**
+ * For parsing a simple (sub)option for a command/major option
+ */
+class OptionSimple extends Option
+{
+
+    final String displayPrefix;
+    final Pattern matchPrefix;
+    final String defaultValue;
+    final Pattern pattern;
+    final String description;
+    final boolean required;
+    String value;
+
+    public OptionSimple(String prefix, String valuePattern, String defaultValue, String description, boolean required)
+    {
+        this.displayPrefix = prefix;
+        this.matchPrefix = Pattern.compile(Pattern.quote(prefix), Pattern.CASE_INSENSITIVE);
+        this.pattern = Pattern.compile(valuePattern, Pattern.CASE_INSENSITIVE);
+        this.defaultValue = defaultValue;
+        this.description = description;
+        this.required = required;
+    }
+
+    public OptionSimple(String displayPrefix, Pattern matchPrefix, Pattern valuePattern, String defaultValue, String description, boolean required)
+    {
+        this.displayPrefix = displayPrefix;
+        this.matchPrefix = matchPrefix;
+        this.pattern = valuePattern;
+        this.defaultValue = defaultValue;
+        this.description = description;
+        this.required = required;
+    }
+
+    public boolean setByUser()
+    {
+        return value != null;
+    }
+
+    public boolean present()
+    {
+        return value != null || defaultValue != null;
+    }
+
+    public String value()
+    {
+        return value != null ? value : defaultValue;
+    }
+
+    public boolean accept(String param)
+    {
+        if (matchPrefix.matcher(param).lookingAt())
+        {
+            if (value != null)
+                throw new IllegalArgumentException("Suboption " + displayPrefix + " has been specified more than once");
+            String v = param.substring(displayPrefix.length());
+            if (!pattern.matcher(v).matches())
+                throw new IllegalArgumentException("Invalid option " + param + "; must match pattern " + pattern);
+            value = v;
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean happy()
+    {
+        return !required || value != null;
+    }
+
+    public String shortDisplay()
+    {
+        StringBuilder sb = new StringBuilder();
+        if (!required)
+            sb.append("[");
+        sb.append(displayPrefix);
+        if (displayPrefix.endsWith("="))
+            sb.append("?");
+        if (displayPrefix.endsWith("<"))
+            sb.append("?");
+        if (displayPrefix.endsWith(">"))
+            sb.append("?");
+        if (!required)
+            sb.append("]");
+        return sb.toString();
+    }
+
+    public String longDisplay()
+    {
+        if (description.equals("") && defaultValue == null && pattern.pattern().equals(""))
+            return null;
+        StringBuilder sb = new StringBuilder();
+        sb.append(displayPrefix);
+        if (displayPrefix.endsWith("="))
+            sb.append("?");
+        if (displayPrefix.endsWith("<"))
+            sb.append("?");
+        if (displayPrefix.endsWith(">"))
+            sb.append("?");
+        if (defaultValue != null)
+        {
+            sb.append(" (default=");
+            sb.append(defaultValue);
+            sb.append(")");
+        }
+        return GroupedOptions.formatLong(sb.toString(), description);
+    }
+
+    public List<String> multiLineDisplay()
+    {
+        return Collections.emptyList();
+    }
+
+    public int hashCode()
+    {
+        return displayPrefix.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return that instanceof OptionSimple && ((OptionSimple) that).displayPrefix.equals(this.displayPrefix);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..b3cca10
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsColumn.java
@@ -0,0 +1,176 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.stress.generatedata.*;
+
+/**
+ * For parsing column options
+ */
+public class SettingsColumn implements Serializable
+{
+
+    public final int maxColumnsPerKey;
+    public final List<ByteBuffer> names;
+    public final String comparator;
+    public final boolean useTimeUUIDComparator;
+    public final int superColumns;
+    public final boolean useSuperColumns;
+    public final boolean variableColumnCount;
+
+    private final DistributionFactory sizeDistribution;
+    private final DistributionFactory countDistribution;
+    private final DataGenFactory dataGenFactory;
+
+    public SettingsColumn(GroupedOptions options)
+    {
+        this((Options) options,
+                options instanceof NameOptions ? (NameOptions) options : null,
+                options instanceof CountOptions ? (CountOptions) options : null
+        );
+    }
+
+    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;
+
+            try
+            {
+                parsed = TypeParser.parse(comparator);
+            }
+            catch (Exception e)
+            {
+                System.err.println(e.getMessage());
+                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.");
+                System.exit(1);
+            }
+        }
+        if (name != null)
+        {
+            assert count == null;
+
+            AbstractType comparator;
+            try
+            {
+                comparator = TypeParser.parse(this.comparator);
+            } catch (Exception e)
+            {
+                throw new IllegalStateException(e);
+            }
+
+            final String[] names = name.name.value().split(",");
+            this.names = new ArrayList<>(names.length);
+
+            for (String columnName : names)
+                this.names.add(comparator.fromString(columnName));
+
+            final int nameCount = this.names.size();
+            countDistribution = new DistributionFactory()
+            {
+                @Override
+                public Distribution get()
+                {
+                    return new DistributionFixed(nameCount);
+                }
+            };
+        }
+        else
+        {
+            this.countDistribution = count.count.get();
+            this.names = null;
+        }
+        maxColumnsPerKey = (int) countDistribution.get().maxValue();
+        variableColumnCount = countDistribution.get().minValue() < maxColumnsPerKey;
+    }
+
+    public RowGen newRowGen()
+    {
+        return new RowGenDistributedSize(dataGenFactory.get(), countDistribution.get(), sizeDistribution.get());
+    }
+
+    // Option Declarations
+
+    private static abstract class Options extends GroupedOptions
+    {
+        final OptionSimple superColumns = new OptionSimple("super=", "[0-9]+", "0", "Number of super columns to use (no super columns used if not specified)", false);
+        final OptionSimple comparator = new OptionSimple("comparator=", "TimeUUIDType|AsciiType|UTF8Type", "AsciiType", "Column Comparator to use", false);
+        final OptionDistribution size = new OptionDistribution("size=", "FIXED(34)");
+        final OptionDataGen generator = new OptionDataGen("data=", "REPEAT(50)");
+    }
+
+    private static final class NameOptions extends Options
+    {
+        final OptionSimple name = new OptionSimple("names=", ".*", null, "Column names", true);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(name, superColumns, comparator, size, generator);
+        }
+    }
+
+    private static final class CountOptions extends Options
+    {
+        final OptionDistribution count = new OptionDistribution("n=", "FIXED(5)");
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(count, superColumns, comparator, size, generator);
+        }
+    }
+
+    // CLI Utility Methods
+
+    static SettingsColumn get(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-col");
+        if (params == null)
+            return new SettingsColumn(new CountOptions());
+
+        GroupedOptions options = GroupedOptions.select(params, new NameOptions(), new CountOptions());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -col options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsColumn(options);
+    }
+
+    static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-col", new NameOptions(), new CountOptions());
+    }
+
+    static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..a996988
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
@@ -0,0 +1,159 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.thrift.ConsistencyLevel;
+
+// Generic command settings - common to read/write/etc
+public class SettingsCommand implements Serializable
+{
+
+    public final Command type;
+    public final long count;
+    public final int tries;
+    public final boolean ignoreErrors;
+    public final ConsistencyLevel consistencyLevel;
+    public final double targetUncertainty;
+    public final int minimumUncertaintyMeasurements;
+    public final int maximumUncertaintyMeasurements;
+
+    public SettingsCommand(Command type, GroupedOptions options)
+    {
+        this(type, (Options) options,
+                options instanceof Count ? (Count) options : null,
+                options instanceof Uncertainty ? (Uncertainty) options : null
+        );
+    }
+
+    public SettingsCommand(Command type, Options options, Count count, Uncertainty uncertainty)
+    {
+        this.type = type;
+        this.tries = Math.max(1, Integer.parseInt(options.retries.value()) + 1);
+        this.ignoreErrors = options.ignoreErrors.setByUser();
+        this.consistencyLevel = ConsistencyLevel.valueOf(options.consistencyLevel.value().toUpperCase());
+        if (count != null)
+        {
+            this.count = Long.parseLong(count.count.value());
+            this.targetUncertainty = -1;
+            this.minimumUncertaintyMeasurements = -1;
+            this.maximumUncertaintyMeasurements = -1;
+        }
+        else
+        {
+            this.count = -1;
+            this.targetUncertainty = Double.parseDouble(uncertainty.uncertainty.value());
+            this.minimumUncertaintyMeasurements = Integer.parseInt(uncertainty.minMeasurements.value());
+            this.maximumUncertaintyMeasurements = Integer.parseInt(uncertainty.maxMeasurements.value());
+        }
+    }
+
+    // Option Declarations
+
+    static abstract class Options extends GroupedOptions
+    {
+        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 consistencyLevel = new OptionSimple("cl=", "ONE|QUORUM|LOCAL_QUORUM|EACH_QUORUM|ALL|ANY", "ONE", "Consistency level to use", false);
+    }
+
+    static class Count extends Options
+    {
+
+        final OptionSimple count = new OptionSimple("n=", "[0-9]+", null, "Number of operations to perform", true);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(count, retries, ignoreErrors, consistencyLevel);
+        }
+    }
+
+    static class Uncertainty extends Options
+    {
+
+        final OptionSimple uncertainty = new OptionSimple("err<", "0\\.[0-9]+", "0.02", "Run until the standard error of the mean is below this fraction", false);
+        final OptionSimple minMeasurements = new OptionSimple("n>", "[0-9]+", "30", "Run at least this many iterations before accepting uncertainty convergence", false);
+        final OptionSimple maxMeasurements = new OptionSimple("n<", "[0-9]+", "200", "Run at most this many iterations before accepting uncertainty convergence", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(uncertainty, minMeasurements, maxMeasurements, retries, ignoreErrors, consistencyLevel);
+        }
+    }
+
+    // CLI Utility Methods
+
+    static SettingsCommand get(Map<String, String[]> clArgs)
+    {
+        for (Command cmd : Command.values())
+        {
+            if (cmd.category == null)
+                continue;
+            final String[] params = clArgs.remove(cmd.toString().toLowerCase());
+            if (params != null)
+            {
+                switch (cmd.category)
+                {
+                    case BASIC:
+                        return build(cmd, params);
+                    case MULTI:
+                        return SettingsCommandMulti.build(cmd, params);
+                    case MIXED:
+                        return SettingsCommandMixed.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.toString().toLowerCase(), new Uncertainty(), new Count());
+    }
+
+    static Runnable helpPrinter(final String type)
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp(type);
+            }
+        };
+    }
+
+    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/2e1e98ad/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
new file mode 100644
index 0000000..995e7d6
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMixed.java
@@ -0,0 +1,184 @@
+package org.apache.cassandra.stress.settings;
+
+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 SettingsCommandMulti
+{
+
+    // 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);
+
+        OptionSimple[] ratiosIn = options.probabilities.ratios;
+        List<Pair<Command, Double>> ratiosOut = new ArrayList<>();
+        for (int i = 0 ; i < ratiosIn.length ; i++)
+        {
+            if (ratiosIn[i] != null && ratiosIn[i].present())
+            {
+                double d = Double.parseDouble(ratiosIn[i].value());
+                if (d > 0)
+                    ratiosOut.add(new Pair<>(Command.values()[i], d));
+            }
+        }
+
+        ratios = ratiosOut;
+        clustering = options.clustering.get();
+
+        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");
+            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;
+        }
+    }
+
+    static final class Options extends GroupedOptions
+    {
+        final SettingsCommandMulti.Options parent;
+        protected Options(SettingsCommandMulti.Options parent)
+        {
+            this.parent = parent;
+        }
+        final OptionDistribution clustering = new OptionDistribution("clustering=", "GAUSSIAN(1..10)");
+        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 SettingsCommandMulti.Options(new Uncertainty())),
+                new Options(new SettingsCommandMulti.Options(new 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 SettingsCommandMulti.Options(new Uncertainty())),
+                new Options(new SettingsCommandMulti.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/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMulti.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMulti.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMulti.java
new file mode 100644
index 0000000..720b0c6
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandMulti.java
@@ -0,0 +1,69 @@
+package org.apache.cassandra.stress.settings;
+
+import java.util.ArrayList;
+import java.util.List;
+
+// Settings common to commands that operate over multiple keys at once
+public class SettingsCommandMulti extends SettingsCommand
+{
+
+    public final int keysAtOnce;
+
+    public SettingsCommandMulti(Command type, Options options)
+    {
+        super(type, options.parent);
+        this.keysAtOnce = Integer.parseInt(options.maxKeys.value());
+    }
+
+    // Option Declarations
+
+    static final class Options extends GroupedOptions
+    {
+        final GroupedOptions parent;
+        Options(GroupedOptions parent)
+        {
+            this.parent = parent;
+        }
+        final OptionSimple maxKeys = new OptionSimple("at-once=", "[0-9]+", "1000", "Number of keys per operation", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            final List<Option> options = new ArrayList<>();
+            options.add(maxKeys);
+            options.addAll(parent.options());
+            return options;
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsCommand 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 SettingsCommandMulti(type, (Options) options);
+    }
+
+    public static void printHelp(Command type)
+    {
+        GroupedOptions.printOptions(System.out, type.toString().toLowerCase(), new Options(new Uncertainty()), new Options(new Count()));
+    }
+
+    public static Runnable helpPrinter(final Command type)
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp(type);
+            }
+        };
+    }
+}


[5/6] Improve Stress Tool patch by Benedict; reviewed by Pavel Yaskevich for CASSANDRA-6199

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java b/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
new file mode 100644
index 0000000..b9f1a47
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
@@ -0,0 +1,178 @@
+package org.apache.cassandra.stress;
+
+import java.io.PrintStream;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadFactory;
+
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.stress.util.Timing;
+import org.apache.cassandra.stress.util.TimingInterval;
+import org.apache.cassandra.stress.util.Uncertainty;
+import org.apache.commons.lang3.time.DurationFormatUtils;
+
+public class StressMetrics
+{
+
+    private static final ThreadFactory tf = new NamedThreadFactory("StressMetrics");
+
+    private final PrintStream output;
+    private final Thread thread;
+    private volatile boolean stop = false;
+    private volatile boolean cancelled = false;
+    private final Uncertainty opRateUncertainty = new Uncertainty();
+    private final CountDownLatch stopped = new CountDownLatch(1);
+    private final Timing timing = new Timing();
+
+    public StressMetrics(PrintStream output, final long logIntervalMillis)
+    {
+        this.output = output;
+        printHeader("", output);
+        thread = tf.newThread(new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                timing.start();
+                try {
+
+                    while (!stop)
+                    {
+                        try
+                        {
+                            long sleep = timing.getHistory().endMillis() + logIntervalMillis - System.currentTimeMillis();
+                            if (sleep < logIntervalMillis >>> 3)
+                                // if had a major hiccup, sleep full interval
+                                Thread.sleep(logIntervalMillis);
+                            else
+                                Thread.sleep(sleep);
+                            update();
+                        } catch (InterruptedException e)
+                        {
+                            break;
+                        }
+                    }
+
+                    update();
+                }
+                catch (InterruptedException e)
+                {}
+                catch (Exception e)
+                {
+                    cancel();
+                    e.printStackTrace(StressMetrics.this.output);
+                }
+                finally
+                {
+                    stopped.countDown();
+                }
+            }
+        });
+    }
+
+    public void start()
+    {
+        thread.start();
+    }
+
+    public void waitUntilConverges(double targetUncertainty, int minMeasurements, int maxMeasurements) throws InterruptedException
+    {
+        opRateUncertainty.await(targetUncertainty, minMeasurements, maxMeasurements);
+    }
+
+    public void cancel()
+    {
+        cancelled = true;
+        stop = true;
+        thread.interrupt();
+        opRateUncertainty.wakeAll();
+    }
+
+    public void stop() throws InterruptedException
+    {
+        stop = true;
+        thread.interrupt();
+        stopped.await();
+    }
+
+    private void update() throws InterruptedException
+    {
+        TimingInterval interval = timing.snapInterval();
+        printRow("", interval, timing.getHistory(), opRateUncertainty, output);
+        opRateUncertainty.update(interval.adjustedOpRate());
+    }
+
+
+    // PRINT FORMATTING
+
+    public static final String HEADFORMAT = "%-10s,%8s,%8s,%8s,%8s,%8s,%8s,%8s,%8s,%8s,%7s,%9s";
+    public static final String ROWFORMAT =  "%-10d,%8.0f,%8.0f,%8.0f,%8.1f,%8.1f,%8.1f,%8.1f,%8.1f,%8.1f,%7.1f,%9.5f";
+
+    private static void printHeader(String prefix, PrintStream output)
+    {
+        output.println(prefix + String.format(HEADFORMAT, "ops","op/s", "adj op/s","key/s","mean","med",".95",".99",".999","max","time","stderr"));
+    }
+
+    private static void printRow(String prefix, TimingInterval interval, TimingInterval total, Uncertainty opRateUncertainty, PrintStream output)
+    {
+        output.println(prefix + String.format(ROWFORMAT,
+                total.operationCount,
+                interval.realOpRate(),
+                interval.adjustedOpRate(),
+                interval.keyRate(),
+                interval.meanLatency(),
+                interval.medianLatency(),
+                interval.rankLatency(0.95f),
+                interval.rankLatency(0.99f),
+                interval.rankLatency(0.999f),
+                interval.maxLatency(),
+                total.runTime() / 1000f,
+                opRateUncertainty.getUncertainty()));
+    }
+
+    public void summarise()
+    {
+        output.println("\n");
+        output.println("Results:");
+        TimingInterval history = timing.getHistory();
+        output.println(String.format("real op rate              : %.0f", history.realOpRate()));
+        output.println(String.format("adjusted op rate          : %.0f", history.adjustedOpRate()));
+        output.println(String.format("adjusted op rate stderr   : %.0f", opRateUncertainty.getUncertainty()));
+        output.println(String.format("key rate                  : %.0f", history.keyRate()));
+        output.println(String.format("latency mean              : %.1f", history.meanLatency()));
+        output.println(String.format("latency median            : %.1f", history.medianLatency()));
+        output.println(String.format("latency 95th percentile   : %.1f", history.rankLatency(.95f)));
+        output.println(String.format("latency 99th percentile   : %.1f", history.rankLatency(0.99f)));
+        output.println(String.format("latency 99.9th percentile : %.1f", history.rankLatency(0.999f)));
+        output.println(String.format("latency max               : %.1f", history.maxLatency()));
+        output.println("Total operation time      : " + DurationFormatUtils.formatDuration(
+                history.runTime(), "HH:mm:ss", true));
+    }
+
+    public static final void summarise(List<String> ids, List<StressMetrics> summarise, PrintStream out)
+    {
+        int idLen = 0;
+        for (String id : ids)
+            idLen = Math.max(id.length(), idLen);
+        String formatstr = "%" + idLen + "s, ";
+        printHeader(String.format(formatstr, "id"), out);
+        for (int i = 0 ; i < ids.size() ; i++)
+            printRow(String.format(formatstr, ids.get(i)),
+                    summarise.get(i).timing.getHistory(),
+                    summarise.get(i).timing.getHistory(),
+                    summarise.get(i).opRateUncertainty,
+                    out
+            );
+    }
+
+    public Timing getTiming()
+    {
+        return timing;
+    }
+
+    public boolean wasCancelled()
+    {
+        return cancelled;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/StressServer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressServer.java b/tools/stress/src/org/apache/cassandra/stress/StressServer.java
index 6600dfd..3c9e2a6 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressServer.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressServer.java
@@ -1,27 +1,30 @@
 /**
- * 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.
- */
+* 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;
 
 import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.PrintStream;
 import java.net.InetAddress;
 import java.net.ServerSocket;
+import java.net.Socket;
 
-import org.apache.cassandra.stress.server.StressThread;
+import org.apache.cassandra.stress.settings.StressSettings;
 import org.apache.commons.cli.*;
 
 public class StressServer
@@ -68,4 +71,57 @@ public class StressServer
         for (;;)
             new StressThread(serverSocket.accept()).start();
     }
+
+    public static class StressThread extends Thread
+    {
+        private final Socket socket;
+
+        public StressThread(Socket client)
+        {
+            this.socket = client;
+        }
+
+        public void run()
+        {
+            try
+            {
+                ObjectInputStream in = new ObjectInputStream(socket.getInputStream());
+                PrintStream out = new PrintStream(socket.getOutputStream());
+
+                StressAction action = new StressAction((StressSettings) in.readObject(), out);
+                Thread actionThread = new Thread(action);
+                actionThread.start();
+
+                while (actionThread.isAlive())
+                {
+                    try
+                    {
+                        if (in.readInt() == 1)
+                        {
+                            actionThread.interrupt();
+                            break;
+                        }
+                    }
+                    catch (Exception e)
+                    {
+                        // continue without problem
+                    }
+                }
+
+                out.close();
+                in.close();
+                socket.close();
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e.getMessage(), e);
+            }
+            catch (Exception e)
+            {
+                e.printStackTrace();
+            }
+        }
+
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/StressStatistics.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressStatistics.java b/tools/stress/src/org/apache/cassandra/stress/StressStatistics.java
deleted file mode 100644
index b739c8e..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/StressStatistics.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/**
- * 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;
-
-import java.io.PrintStream;
-import org.apache.commons.lang3.time.DurationFormatUtils;
-
-import com.yammer.metrics.stats.Snapshot;
-
-
-/**
- * Gathers and aggregates statistics for an operation
- */
-public class StressStatistics
-{
-    
-    private Session client;
-    private PrintStream output;
-
-    private long durationInSeconds;
-    /** The sum of the interval_op_rate values collected by tallyAverages */
-    private int tallyOpRateSum;
-    /** The number of interval_op_rate values collected by tallyAverages */
-    private int tallyOpRateCount;
-    /** The sum of the interval_key_rate values collected by tallyAverages */
-    private int tallyKeyRateSum;
-    /** The number of interval_key_rate values collected by tallyAverages */
-    private int tallyKeyRateCount;
-
-    /** The sum of the latency values collected by tallyAverages */
-    private double tallyLatencySum;
-    /** The number of latency values collected by tallyAverages */
-    private int tallyLatencyCount;
-    /** The sum of the 95%tile latency values collected by tallyAverages */
-    private double tally95thLatencySum;
-    /** The number of 95%tile latency values collected by tallyAverages */
-    private int tally95thLatencyCount;
-    /** The sum of the 99.9%tile latency values collected by tallyAverages */
-    private double tally999thLatencySum;
-    /** The number of 99.9%tile latency values collected by tallyAverages */
-    private int tally999thLatencyCount;
-    
-
-    public StressStatistics(Session client, PrintStream out)
-    {
-        this.client = client;
-        this.output = out;
-
-        tallyOpRateSum = 0;
-        tallyOpRateCount = 0;
-    }
-
-    /**
-     * Collect statistics per-interval
-     */
-    public void addIntervalStats(int totalOperations, int intervalOpRate, 
-                                 int intervalKeyRate, Snapshot latency, 
-                                 long currentTimeInSeconds)
-    {
-        this.tallyAverages(totalOperations, intervalKeyRate, intervalKeyRate, 
-                                latency, currentTimeInSeconds);
-    }
-
-    /**
-     * Collect interval_op_rate and interval_key_rate averages
-     */
-    private void tallyAverages(int totalOperations, int intervalOpRate, 
-                                 int intervalKeyRate, Snapshot latency, 
-                                 long currentTimeInSeconds)
-    {
-        //Skip the first and last 10% of values.
-        //The middle values of the operation are the ones worthwhile
-        //to collect and average:
-        if (totalOperations > (0.10 * client.getNumKeys()) &&
-            totalOperations < (0.90 * client.getNumKeys())) {
-                tallyOpRateSum += intervalOpRate;
-                tallyOpRateCount += 1;
-                tallyKeyRateSum += intervalKeyRate;
-                tallyKeyRateCount += 1;
-                tallyLatencySum += latency.getMedian();
-                tallyLatencyCount += 1;
-                tally95thLatencySum += latency.get95thPercentile();
-                tally95thLatencyCount += 1;
-                tally999thLatencySum += latency.get999thPercentile();
-                tally999thLatencyCount += 1;
-            }
-        durationInSeconds = currentTimeInSeconds;
-    }
-
-    public void printStats()
-    {
-        output.println("\n");
-        if (tallyOpRateCount > 0) {
-            output.println("Averages from the middle 80% of values:");
-            output.println(String.format("interval_op_rate          : %d", 
-                                         (tallyOpRateSum / tallyOpRateCount)));
-            output.println(String.format("interval_key_rate         : %d", 
-                                         (tallyKeyRateSum / tallyKeyRateCount)));
-            output.println(String.format("latency median            : %.1f", 
-                                         (tallyLatencySum / tallyLatencyCount)));
-            output.println(String.format("latency 95th percentile   : %.1f",
-                                         (tally95thLatencySum / tally95thLatencyCount)));
-            output.println(String.format("latency 99.9th percentile : %.1f", 
-                                         (tally999thLatencySum / tally999thLatencyCount)));
-        }
-        output.println("Total operation time      : " + DurationFormatUtils.formatDuration(
-            durationInSeconds*1000, "HH:mm:ss", true));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..4c22005
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGen.java
@@ -0,0 +1,18 @@
+package org.apache.cassandra.stress.generatedata;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public abstract class DataGen
+{
+
+    public abstract void generate(ByteBuffer fill, long offset);
+    public abstract boolean isDeterministic();
+
+    public void generate(List<ByteBuffer> fills, long offset)
+    {
+        for (ByteBuffer fill : fills)
+            generate(fill, offset++);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..3906f93
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenBytesRandom.java
@@ -0,0 +1,24 @@
+package org.apache.cassandra.stress.generatedata;
+
+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 offset)
+    {
+        fill.clear();
+        rnd.nextBytes(fill.array());
+    }
+
+    @Override
+    public boolean isDeterministic()
+    {
+        return false;
+    }
+
+}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..50d49dd
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHex.java
@@ -0,0 +1,39 @@
+package org.apache.cassandra.stress.generatedata;
+
+import java.nio.ByteBuffer;
+
+public abstract class DataGenHex extends DataGen
+{
+
+    abstract long next(long operationIndex);
+
+    @Override
+    public final void generate(ByteBuffer fill, long operationIndex)
+    {
+        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/2e1e98ad/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
new file mode 100644
index 0000000..3391fce
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromDistribution.java
@@ -0,0 +1,45 @@
+package org.apache.cassandra.stress.generatedata;
+
+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/2e1e98ad/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
new file mode 100644
index 0000000..5d499d5
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenHexFromOpIndex.java
@@ -0,0 +1,27 @@
+package org.apache.cassandra.stress.generatedata;
+
+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/2e1e98ad/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
new file mode 100644
index 0000000..68c8034
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringDictionary.java
@@ -0,0 +1,84 @@
+package org.apache.cassandra.stress.generatedata;
+
+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)
+    {
+        fill(fill, 0);
+    }
+
+    @Override
+    public void generate(List<ByteBuffer> fills, long index)
+    {
+        for (int i = 0 ; i < fills.size() ; i++)
+            fill(fills.get(0), i);
+    }
+
+    private void fill(ByteBuffer fill, int column)
+    {
+        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 true;
+    }
+
+    public static DataGenFactory getFactory(File file) throws IOException
+    {
+        final List<Pair<byte[], Double>> words = new ArrayList<>();
+        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/2e1e98ad/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
new file mode 100644
index 0000000..47091f7
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DataGenStringRepeats.java
@@ -0,0 +1,69 @@
+package org.apache.cassandra.stress.generatedata;
+
+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)
+    {
+        fill(fill, index, 0);
+    }
+
+    @Override
+    public void generate(List<ByteBuffer> fills, long index)
+    {
+        for (int i = 0 ; i < fills.size() ; i++)
+        {
+            fill(fills.get(i), index, i);
+        }
+    }
+
+    private void fill(ByteBuffer fill, long index, int column)
+    {
+        fill.clear();
+        byte[] trg = fill.array();
+        byte[] src = getData(index, column);
+        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)
+    {
+        final long key = (column * repeatFrequency) + (index % 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/2e1e98ad/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
new file mode 100644
index 0000000..5236eab
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/Distribution.java
@@ -0,0 +1,19 @@
+package org.apache.cassandra.stress.generatedata;
+
+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);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..9f59dbd
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionBoundApache.java
@@ -0,0 +1,42 @@
+package org.apache.cassandra.stress.generatedata;
+
+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();
+    }
+
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..6873b1c
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionFixed.java
@@ -0,0 +1,25 @@
+package org.apache.cassandra.stress.generatedata;
+
+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;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionOffsetApache.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionOffsetApache.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionOffsetApache.java
new file mode 100644
index 0000000..c7a5aca
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionOffsetApache.java
@@ -0,0 +1,40 @@
+package org.apache.cassandra.stress.generatedata;
+
+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;
+    }
+
+    @Override
+    public long next()
+    {
+        return offset(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;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionSeqBatch.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionSeqBatch.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionSeqBatch.java
new file mode 100644
index 0000000..a1a51bb
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/DistributionSeqBatch.java
@@ -0,0 +1,47 @@
+package org.apache.cassandra.stress.generatedata;
+
+public class DistributionSeqBatch extends DataGenHex
+{
+
+    final Distribution delegate;
+    final int batchSize;
+    final long maxKey;
+
+    private int batchIndex;
+    private long batchKey;
+
+    // object must be published safely if passed between threadCount, due to batchIndex not being volatile. various
+    // hacks possible, but not ideal. don't want to use volatile as object intended for single threaded use.
+    public DistributionSeqBatch(int batchSize, long maxKey, Distribution delegate)
+    {
+        this.batchIndex = batchSize;
+        this.batchSize = batchSize;
+        this.maxKey = maxKey;
+        this.delegate = delegate;
+    }
+
+    @Override
+    long next(long operationIndex)
+    {
+        if (batchIndex >= batchSize)
+        {
+            batchKey = delegate.next();
+            batchIndex = 0;
+        }
+        long r = batchKey + batchIndex++;
+        if (r > maxKey)
+        {
+            batchKey = delegate.next();
+            batchIndex = 1;
+            r = batchKey;
+        }
+        return r;
+    }
+
+    @Override
+    public boolean isDeterministic()
+    {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/generatedata/KeyGen.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/KeyGen.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/KeyGen.java
new file mode 100644
index 0000000..cdd6d39
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/KeyGen.java
@@ -0,0 +1,33 @@
+package org.apache.cassandra.stress.generatedata;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class KeyGen
+{
+
+    final DataGen dataGen;
+    final int keySize;
+    final List<ByteBuffer> keyBuffers = new ArrayList<>();
+
+    public KeyGen(DataGen dataGen, int keySize)
+    {
+        this.dataGen = dataGen;
+        this.keySize = keySize;
+    }
+
+    public List<ByteBuffer> getKeys(int n, long index)
+    {
+        while (keyBuffers.size() < n)
+            keyBuffers.add(ByteBuffer.wrap(new byte[keySize]));
+        dataGen.generate(keyBuffers, index);
+        return keyBuffers;
+    }
+
+    public boolean isDeterministic()
+    {
+        return dataGen.isDeterministic();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/generatedata/RowGen.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/RowGen.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/RowGen.java
new file mode 100644
index 0000000..869fbc7
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/RowGen.java
@@ -0,0 +1,31 @@
+package org.apache.cassandra.stress.generatedata;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Generates a row of data, by constructing one byte buffers per column according to some algorithm
+ * and delegating the work of populating the values of those byte buffers to the provided data generator
+ */
+public abstract class RowGen
+{
+
+    final DataGen dataGen;
+    protected RowGen(DataGen dataGenerator)
+    {
+        this.dataGen = dataGenerator;
+    }
+
+    public List<ByteBuffer> generate(long operationIndex)
+    {
+        List<ByteBuffer> fill = getColumns(operationIndex);
+        dataGen.generate(fill, operationIndex);
+        return fill;
+    }
+
+    // these byte[] may be re-used
+    abstract List<ByteBuffer> getColumns(long operationIndex);
+
+    abstract public boolean isDeterministic();
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/generatedata/RowGenDistributedSize.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generatedata/RowGenDistributedSize.java b/tools/stress/src/org/apache/cassandra/stress/generatedata/RowGenDistributedSize.java
new file mode 100644
index 0000000..b68ab3c
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generatedata/RowGenDistributedSize.java
@@ -0,0 +1,84 @@
+package org.apache.cassandra.stress.generatedata;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class RowGenDistributedSize extends RowGen
+{
+
+    // TODO - make configurable
+    static final int MAX_SINGLE_CACHE_SIZE = 16 * 1024;
+
+    final Distribution countDistribution;
+    final Distribution sizeDistribution;
+
+    final TreeMap<Integer, ByteBuffer> cache = new TreeMap<>();
+
+    // array re-used for returning columns
+    final ByteBuffer[] ret;
+    final int[] sizes;
+
+    public RowGenDistributedSize(DataGen dataGenerator, Distribution countDistribution, Distribution sizeDistribution)
+    {
+        super(dataGenerator);
+        this.countDistribution = countDistribution;
+        this.sizeDistribution = sizeDistribution;
+        ret = new ByteBuffer[(int) countDistribution.maxValue()];
+        sizes = new int[ret.length];
+    }
+
+    ByteBuffer getBuffer(int size)
+    {
+        if (size >= MAX_SINGLE_CACHE_SIZE)
+            return ByteBuffer.allocate(size);
+        Map.Entry<Integer, ByteBuffer> found = cache.ceilingEntry(size);
+        if (found == null)
+        {
+            // remove the next entry down, and replace it with a cache of this size
+            Integer del = cache.lowerKey(size);
+            if (del != null)
+                cache.remove(del);
+            return ByteBuffer.allocate(size);
+        }
+        ByteBuffer r = found.getValue();
+        cache.remove(found.getKey());
+        return r;
+    }
+
+    @Override
+    List<ByteBuffer> getColumns(long operationIndex)
+    {
+        int i = 0;
+        int count = (int) countDistribution.next();
+        while (i < count)
+        {
+            int columnSize = (int) sizeDistribution.next();
+            sizes[i] = columnSize;
+            ret[i] = getBuffer(columnSize);
+            i++;
+        }
+        while (i < ret.length && ret[i] != null)
+            ret[i] = null;
+        i = 0;
+        while (i < count)
+        {
+            ByteBuffer b = ret[i];
+            cache.put(b.capacity(), b);
+            b.position(b.capacity() - sizes[i]);
+            ret[i] = b.slice();
+            b.position(0);
+            i++;
+        }
+        return Arrays.asList(ret).subList(0, count);
+    }
+
+    @Override
+    public boolean isDeterministic()
+    {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CQLOperation.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CQLOperation.java b/tools/stress/src/org/apache/cassandra/stress/operations/CQLOperation.java
deleted file mode 100644
index 54737a4..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CQLOperation.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.operations;
-
-import java.nio.ByteBuffer;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.transport.SimpleClient;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.thrift.Compression;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.thrift.ThriftConversion;
-
-public abstract class CQLOperation extends Operation
-{
-    public CQLOperation(Session client, int idx)
-    {
-        super(client, idx);
-    }
-
-    protected abstract void run(CQLQueryExecutor executor) throws IOException;
-
-    protected abstract boolean validateThriftResult(CqlResult result);
-
-    protected abstract boolean validateNativeResult(ResultMessage result);
-
-    public void run(final CassandraClient client) throws IOException
-    {
-        run(new CQLQueryExecutor()
-        {
-            public boolean execute(String cqlQuery, List<String> queryParams) throws Exception
-            {
-                CqlResult result = null;
-                if (session.usePreparedStatements())
-                {
-                    Integer stmntId = getPreparedStatement(client, cqlQuery);
-                    if (session.cqlVersion.startsWith("3"))
-                        result = client.execute_prepared_cql3_query(stmntId, queryParamsAsByteBuffer(queryParams), session.getConsistencyLevel());
-                    else
-                        result = client.execute_prepared_cql_query(stmntId, queryParamsAsByteBuffer(queryParams));
-                }
-                else
-                {
-                    String formattedQuery = formatCqlQuery(cqlQuery, queryParams);
-                    if (session.cqlVersion.startsWith("3"))
-                        result = client.execute_cql3_query(ByteBuffer.wrap(formattedQuery.getBytes()), Compression.NONE, session.getConsistencyLevel());
-                    else
-                        result = client.execute_cql_query(ByteBuffer.wrap(formattedQuery.getBytes()), Compression.NONE);
-                }
-                return validateThriftResult(result);
-            }
-        });
-    }
-
-    public void run(final SimpleClient client) throws IOException
-    {
-        run(new CQLQueryExecutor()
-        {
-            public boolean execute(String cqlQuery, List<String> queryParams) throws Exception
-            {
-                ResultMessage result = null;
-                if (session.usePreparedStatements())
-                {
-                    byte[] stmntId = getPreparedStatement(client, cqlQuery);
-                    result = client.executePrepared(stmntId, queryParamsAsByteBuffer(queryParams), ThriftConversion.fromThrift(session.getConsistencyLevel()));
-                }
-                else
-                {
-                    String formattedQuery = formatCqlQuery(cqlQuery, queryParams);
-                    result = client.execute(formattedQuery, ThriftConversion.fromThrift(session.getConsistencyLevel()));
-                }
-                return validateNativeResult(result);
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CounterAdder.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CounterAdder.java b/tools/stress/src/org/apache/cassandra/stress/operations/CounterAdder.java
deleted file mode 100644
index ab6ae9d..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CounterAdder.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * 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.operations;
-
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class CounterAdder extends Operation
-{
-    public CounterAdder(Session client, int index)
-    {
-        super(client, index);
-    }
-
-    public void run(CassandraClient client) throws IOException
-    {
-        List<CounterColumn> columns = new ArrayList<CounterColumn>();
-        List<CounterSuperColumn> superColumns = new ArrayList<CounterSuperColumn>();
-
-        // format used for keys
-        String format = "%0" + session.getTotalKeysLength() + "d";
-
-        for (int i = 0; i < session.getColumnsPerKey(); i++)
-        {
-            String columnName = ("C" + Integer.toString(i));
-
-            columns.add(new CounterColumn(ByteBufferUtil.bytes(columnName), 1L));
-        }
-
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-        {
-            // supers = [SuperColumn('S' + str(j), columns) for j in xrange(supers_per_key)]
-            for (int i = 0; i < session.getSuperColumns(); i++)
-            {
-                String superColumnName = "S" + Integer.toString(i);
-                superColumns.add(new CounterSuperColumn(ByteBuffer.wrap(superColumnName.getBytes()), columns));
-            }
-        }
-
-        String rawKey = String.format(format, index);
-        Map<ByteBuffer, Map<String, List<Mutation>>> record = new HashMap<ByteBuffer, Map<String, List<Mutation>>>();
-
-        record.put(ByteBufferUtil.bytes(rawKey), session.getColumnFamilyType() == ColumnFamilyType.Super
-                                                                                ? getSuperColumnsMutationMap(superColumns)
-                                                                                : getColumnsMutationMap(columns));
-
-        TimerContext context = session.latency.time();
-
-        boolean success = false;
-        String exceptionMessage = null;
-
-        for (int t = 0; t < session.getRetryTimes(); t++)
-        {
-            if (success)
-                break;
-
-            try
-            {
-                client.batch_mutate(record, session.getConsistencyLevel());
-                success = true;
-            }
-            catch (Exception e)
-            {
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
-            }
-        }
-
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error incrementing key %s %s%n",
-                                index,
-                                session.getRetryTimes(),
-                                rawKey,
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-        }
-
-        session.operations.getAndIncrement();
-        session.keys.getAndIncrement();
-        context.stop();
-    }
-
-    private Map<String, List<Mutation>> getSuperColumnsMutationMap(List<CounterSuperColumn> superColumns)
-    {
-        List<Mutation> mutations = new ArrayList<Mutation>();
-        Map<String, List<Mutation>> mutationMap = new HashMap<String, List<Mutation>>();
-
-        for (CounterSuperColumn s : superColumns)
-        {
-            ColumnOrSuperColumn cosc = new ColumnOrSuperColumn().setCounter_super_column(s);
-            mutations.add(new Mutation().setColumn_or_supercolumn(cosc));
-        }
-
-        mutationMap.put("SuperCounter1", mutations);
-
-        return mutationMap;
-    }
-
-    private Map<String, List<Mutation>> getColumnsMutationMap(List<CounterColumn> columns)
-    {
-        List<Mutation> mutations = new ArrayList<Mutation>();
-        Map<String, List<Mutation>> mutationMap = new HashMap<String, List<Mutation>>();
-
-        for (CounterColumn c : columns)
-        {
-            ColumnOrSuperColumn cosc = new ColumnOrSuperColumn().setCounter_column(c);
-            mutations.add(new Mutation().setColumn_or_supercolumn(cosc));
-        }
-
-        mutationMap.put("Counter1", mutations);
-
-        return mutationMap;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CounterGetter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CounterGetter.java b/tools/stress/src/org/apache/cassandra/stress/operations/CounterGetter.java
deleted file mode 100644
index 56ef243..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CounterGetter.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/**
- * 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.operations;
-
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.thrift.*;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-public class CounterGetter extends Operation
-{
-    public CounterGetter(Session client, int index)
-    {
-        super(client, index);
-    }
-
-    public void run(CassandraClient client) throws IOException
-    {
-        SliceRange sliceRange = new SliceRange();
-
-        // start/finish
-        sliceRange.setStart(new byte[] {}).setFinish(new byte[] {});
-
-        // reversed/count
-        sliceRange.setReversed(false).setCount(session.getColumnsPerKey());
-
-        // initialize SlicePredicate with existing SliceRange
-        SlicePredicate predicate = new SlicePredicate().setSlice_range(sliceRange);
-
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-        {
-            runSuperCounterGetter(predicate, client);
-        }
-        else
-        {
-            runCounterGetter(predicate, client);
-        }
-    }
-
-    private void runSuperCounterGetter(SlicePredicate predicate, Cassandra.Client client) throws IOException
-    {
-        byte[] rawKey = generateKey();
-        ByteBuffer key = ByteBuffer.wrap(rawKey);
-
-        for (int j = 0; j < session.getSuperColumns(); j++)
-        {
-            String superColumn = 'S' + Integer.toString(j);
-            ColumnParent parent = new ColumnParent("SuperCounter1").setSuper_column(superColumn.getBytes());
-
-            TimerContext context = session.latency.time();
-
-            boolean success = false;
-            String exceptionMessage = null;
-
-            for (int t = 0; t < session.getRetryTimes(); t++)
-            {
-                if (success)
-                    break;
-
-                try
-                {
-                    List<ColumnOrSuperColumn> counters;
-                    counters = client.get_slice(key, parent, predicate, session.getConsistencyLevel());
-                    success = (counters.size() != 0);
-                }
-                catch (Exception e)
-                {
-                    exceptionMessage = getExceptionMessage(e);
-                    success = false;
-                }
-            }
-
-            if (!success)
-            {
-                error(String.format("Operation [%d] retried %d times - error reading counter key %s %s%n",
-                                    index,
-                                    session.getRetryTimes(),
-                                    new String(rawKey),
-                                    (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-            }
-
-            session.operations.getAndIncrement();
-            session.keys.getAndIncrement();
-            context.stop();
-        }
-    }
-
-    private void runCounterGetter(SlicePredicate predicate, Cassandra.Client client) throws IOException
-    {
-        ColumnParent parent = new ColumnParent("Counter1");
-
-        byte[] key = generateKey();
-        ByteBuffer keyBuffer = ByteBuffer.wrap(key);
-
-        TimerContext context = session.latency.time();
-
-        boolean success = false;
-        String exceptionMessage = null;
-
-        for (int t = 0; t < session.getRetryTimes(); t++)
-        {
-            if (success)
-                break;
-
-            try
-            {
-                List<ColumnOrSuperColumn> counters;
-                counters = client.get_slice(keyBuffer, parent, predicate, session.getConsistencyLevel());
-                success = (counters.size() != 0);
-            }
-            catch (Exception e)
-            {
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
-            }
-        }
-
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error reading counter key %s %s%n",
-                                index,
-                                session.getRetryTimes(),
-                                new String(key),
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-        }
-
-        session.operations.getAndIncrement();
-        session.keys.getAndIncrement();
-        context.stop();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java
index 31e8371..8e1f137 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java
@@ -21,102 +21,50 @@ package org.apache.cassandra.stress.operations;
  */
 
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.thrift.Compression;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class CqlCounterAdder extends CQLOperation
+public class CqlCounterAdder extends CqlOperation<Integer>
 {
-    private static String cqlQuery = null;
-
-    public CqlCounterAdder(Session client, int idx)
+    public CqlCounterAdder(State state, long idx)
     {
-        super(client, idx);
+        super(state, idx);
     }
 
-    protected void run(CQLQueryExecutor executor) throws IOException
+    @Override
+    protected String buildQuery()
     {
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-            throw new RuntimeException("Super columns are not implemented for CQL");
-
-        if (cqlQuery == null)
-        {
-            String counterCF = session.cqlVersion.startsWith("2") ? "Counter1" : "Counter3";
-
-            StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotesIfRequired(counterCF));
-
-            if (session.cqlVersion.startsWith("2"))
-                query.append(" USING CONSISTENCY ").append(session.getConsistencyLevel());
-
-            query.append(" SET ");
+        String counterCF = state.isCql2() ? "Counter1" : "Counter3";
 
-            for (int i = 0; i < session.getColumnsPerKey(); i++)
-            {
-                if (i > 0)
-                    query.append(",");
+        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotesIfRequired(counterCF));
 
-                query.append('C').append(i).append("=C").append(i).append("+1");
-            }
-            query.append(" WHERE KEY=?");
-            cqlQuery = query.toString();
-        }
-
-        String key = String.format("%0" + session.getTotalKeysLength() + "d", index);
-        List<String> queryParams = Collections.singletonList(getUnQuotedCqlBlob(key, session.cqlVersion.startsWith("3")));
+        if (state.isCql2())
+            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
 
-        TimerContext context = session.latency.time();
+        query.append(" SET ");
 
-        boolean success = false;
-        String exceptionMessage = null;
-
-        for (int t = 0; t < session.getRetryTimes(); t++)
+        // TODO : increment distribution subset of columns
+        for (int i = 0; i < state.settings.columns.maxColumnsPerKey; i++)
         {
-            if (success)
-                break;
+            if (i > 0)
+                query.append(",");
 
-            try
-            {
-                success = executor.execute(cqlQuery, queryParams);
-            }
-            catch (Exception e)
-            {
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
-            }
+            query.append('C').append(i).append("=C").append(i).append("+1");
         }
-
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error incrementing key %s %s%n",
-                                index,
-                                session.getRetryTimes(),
-                                key,
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-        }
-
-        session.operations.getAndIncrement();
-        session.keys.getAndIncrement();
-        context.stop();
+        query.append(" WHERE KEY=?");
+        return query.toString();
     }
 
-    protected boolean validateThriftResult(CqlResult result)
+    @Override
+    protected List<ByteBuffer> getQueryParameters(byte[] key)
     {
-        return true;
+        return Collections.singletonList(ByteBuffer.wrap(key));
     }
 
-    protected boolean validateNativeResult(ResultMessage result)
+    @Override
+    protected CqlRunOp buildRunOp(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String keyid, ByteBuffer key)
     {
-        return true;
+        return new CqlRunOpAlwaysSucceed(client, query, queryId, params, keyid, key, 1);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java
index a4d037a..0a0b05b 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java
@@ -21,100 +21,48 @@ package org.apache.cassandra.stress.operations;
  */
 
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.thrift.Compression;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.thrift.CqlResultType;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class CqlCounterGetter extends CQLOperation
+public class CqlCounterGetter extends CqlOperation<Integer>
 {
-    private static String cqlQuery = null;
 
-    public CqlCounterGetter(Session client, int idx)
+    public CqlCounterGetter(State state, long idx)
     {
-        super(client, idx);
+        super(state, idx);
     }
 
-    protected void run(CQLQueryExecutor executor) throws IOException
+    @Override
+    protected List<ByteBuffer> getQueryParameters(byte[] key)
     {
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-            throw new RuntimeException("Super columns are not implemented for CQL");
-
-        if (cqlQuery == null)
-        {
-            StringBuilder query = new StringBuilder("SELECT ");
-
-            if (session.cqlVersion.startsWith("2"))
-                query.append("FIRST ").append(session.getColumnsPerKey()).append(" ''..''");
-            else
-                query.append("*");
-
-            String counterCF = session.cqlVersion.startsWith("2") ? "Counter1" : "Counter3";
-
-            query.append(" FROM ").append(wrapInQuotesIfRequired(counterCF));
-
-            if (session.cqlVersion.startsWith("2"))
-                query.append(" USING CONSISTENCY ").append(session.getConsistencyLevel().toString());
-
-            cqlQuery = query.append(" WHERE KEY=?").toString();
-        }
-
-        byte[] key = generateKey();
-        List<String> queryParams = Collections.singletonList(getUnQuotedCqlBlob(key, session.cqlVersion.startsWith("3")));
+        return Collections.singletonList(ByteBuffer.wrap(key));
+    }
 
-        TimerContext context = session.latency.time();
+    @Override
+    protected String buildQuery()
+    {
+        StringBuilder query = new StringBuilder("SELECT ");
 
-        boolean success = false;
-        String exceptionMessage = null;
+        if (state.isCql2())
+            query.append("FIRST ").append(state.settings.columns.maxColumnsPerKey).append(" ''..''");
+        else
+            query.append("*");
 
-        for (int t = 0; t < session.getRetryTimes(); t++)
-        {
-            if (success)
-                break;
+        String counterCF = state.isCql2() ? "Counter1" : "Counter3";
 
-            try
-            {
-                success = executor.execute(cqlQuery, queryParams);
-            }
-            catch (Exception e)
-            {
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
-            }
-        }
+        query.append(" FROM ").append(wrapInQuotesIfRequired(counterCF));
 
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error reading counter key %s %s%n",
-                                index,
-                                session.getRetryTimes(),
-                                new String(key),
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-        }
+        if (state.isCql2())
+            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
 
-        session.operations.getAndIncrement();
-        session.keys.getAndIncrement();
-        context.stop();
+        return query.append(" WHERE KEY=?").toString();
     }
 
-    protected boolean validateThriftResult(CqlResult result)
+    @Override
+    protected CqlRunOp buildRunOp(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String keyid, ByteBuffer key)
     {
-        return result.rows.get(0).columns.size() != 0;
+        return new CqlRunOpTestNonEmpty(client, query, queryId, params, keyid, key);
     }
 
-    protected boolean validateNativeResult(ResultMessage result)
-    {
-        return result instanceof ResultMessage.Rows && ((ResultMessage.Rows)result).result.size() != 0;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java
index bf416cc..748bf30 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java
@@ -1,179 +1,123 @@
 package org.apache.cassandra.stress.operations;
 /*
- *
- * 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.
- *
- */
+*
+* 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.IOException;
 import java.nio.ByteBuffer;
-import java.util.Collections;
+import java.util.Arrays;
 import java.util.List;
 
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.cql3.ResultSet;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.thrift.Compression;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.thrift.CqlRow;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class CqlIndexedRangeSlicer extends CQLOperation
+import org.apache.cassandra.stress.settings.SettingsCommandMulti;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class CqlIndexedRangeSlicer extends CqlOperation<byte[][]>
 {
-    private static List<ByteBuffer> values = null;
-    private static String cqlQuery = null;
 
-    private int lastQueryResultSize;
-    private int lastMaxKey;
+    volatile boolean acceptNoResults = false;
 
-    public CqlIndexedRangeSlicer(Session client, int idx)
+    public CqlIndexedRangeSlicer(State state, long idx)
     {
-        super(client, idx);
+        super(state, idx);
     }
 
-    protected void run(CQLQueryExecutor executor) throws IOException
+    @Override
+    protected List<ByteBuffer> getQueryParameters(byte[] key)
     {
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-            throw new RuntimeException("Super columns are not implemented for CQL");
-
-        if (values == null)
-            values = generateValues();
-
-        if (cqlQuery == null)
-        {
-            StringBuilder query = new StringBuilder("SELECT ");
-
-            if (session.cqlVersion.startsWith("2"))
-                query.append(session.getColumnsPerKey()).append(" ''..''");
-            else
-                query.append("*");
-
-            query.append(" FROM Standard1");
-
-            if (session.cqlVersion.startsWith("2"))
-                query.append(" USING CONSISTENCY ").append(session.getConsistencyLevel());
+        throw new UnsupportedOperationException();
+    }
 
-            query.append(" WHERE C1=").append(getUnQuotedCqlBlob(values.get(1).array(), session.cqlVersion.startsWith("3")))
-                 .append(" AND KEY > ? LIMIT ").append(session.getKeysPerCall());
+    @Override
+    protected String buildQuery()
+    {
+        StringBuilder query = new StringBuilder("SELECT ");
 
-            cqlQuery = query.toString();
-        }
+        if (state.isCql2())
+            query.append(state.settings.columns.maxColumnsPerKey).append(" ''..''");
+        else
+            query.append("*");
 
-        String format = "%0" + session.getTotalKeysLength() + "d";
-        String startOffset = String.format(format, 0);
+        query.append(" FROM Standard1");
 
-        int expectedPerValue = session.getNumKeys() / values.size(), received = 0;
+        if (state.isCql2())
+            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
 
-        while (received < expectedPerValue)
-        {
-            TimerContext context = session.latency.time();
-
-            boolean success = false;
-            String exceptionMessage = null;
-            String formattedQuery = null;
-            List<String> queryParms = Collections.singletonList(getUnQuotedCqlBlob(startOffset, session.cqlVersion.startsWith("3")));
-
-            for (int t = 0; t < session.getRetryTimes(); t++)
-            {
-                if (success)
-                    break;
-
-                try
-                {
-                    success = executor.execute(cqlQuery, queryParms);
-                }
-                catch (Exception e)
-                {
-                    exceptionMessage = getExceptionMessage(e);
-                    success = false;
-                }
-            }
-
-            if (!success)
-            {
-                error(String.format("Operation [%d] retried %d times - error executing indexed range query with offset %s %s%n",
-                                    index,
-                                    session.getRetryTimes(),
-                                    startOffset,
-                                    (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-            }
-
-            received += lastQueryResultSize;
-
-            // convert max key found back to an integer, and increment it
-            startOffset = String.format(format, (1 + lastMaxKey));
-
-            session.operations.getAndIncrement();
-            session.keys.getAndAdd(lastQueryResultSize);
-            context.stop();
-        }
+        final String columnName = getColumnName(1);
+        query.append(" WHERE ").append(columnName).append("=?")
+                .append(" AND KEY > ? LIMIT ").append(((SettingsCommandMulti)state.settings.command).keysAtOnce);
+        return query.toString();
     }
 
-    /**
-     * Get maximum key from CqlRow list
-     * @param rows list of the CqlRow objects
-     * @return maximum key value of the list
-     */
-    private int getMaxKey(List<CqlRow> rows)
+    @Override
+    protected void run(CqlOperation.ClientWrapper client) throws IOException
     {
-        int maxKey = ByteBufferUtil.toInt(rows.get(0).key);
-
-        for (CqlRow row : rows)
+        acceptNoResults = false;
+        final List<ByteBuffer> columns = generateColumnValues();
+        final ByteBuffer value = columns.get(1); // only C1 column is indexed
+        byte[] minKey = new byte[0];
+        int rowCount;
+        do
         {
-            int currentKey = ByteBufferUtil.toInt(row.key);
-            if (currentKey > maxKey)
-                maxKey = currentKey;
-        }
-
-        return maxKey;
+            List<ByteBuffer> params = Arrays.asList(value, ByteBuffer.wrap(minKey));
+            CqlRunOp<byte[][]> op = run(client, params, value, new String(value.array()));
+            byte[][] keys = op.result;
+            rowCount = keys.length;
+            minKey = getNextMinKey(minKey, keys);
+            acceptNoResults = true;
+        } while (rowCount > 0);
     }
 
-    private int getMaxKey(ResultSet rs)
+    private final class IndexedRangeSliceRunOp extends CqlRunOpFetchKeys
     {
-        int maxKey = ByteBufferUtil.toInt(rs.rows.get(0).get(0));
 
-        for (List<ByteBuffer> row : rs.rows)
+        protected IndexedRangeSliceRunOp(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String keyid, ByteBuffer key)
         {
-            int currentKey = ByteBufferUtil.toInt(row.get(0));
-            if (currentKey > maxKey)
-                maxKey = currentKey;
+            super(client, query, queryId, params, keyid, key);
         }
 
-        return maxKey;
+        @Override
+        public boolean validate(byte[][] result)
+        {
+            return acceptNoResults || result.length > 0;
+        }
     }
 
-    protected boolean validateThriftResult(CqlResult result)
+    @Override
+    protected CqlRunOp<byte[][]> buildRunOp(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String keyid, ByteBuffer key)
     {
-        lastQueryResultSize = result.rows.size();
-        lastMaxKey = getMaxKey(result.rows);
-        return lastQueryResultSize != 0;
+        return new IndexedRangeSliceRunOp(client, query, queryId, params, keyid, key);
     }
 
-    protected boolean validateNativeResult(ResultMessage result)
+    private static byte[] getNextMinKey(byte[] cur, byte[][] keys)
     {
-        assert result instanceof ResultMessage.Rows;
-        lastQueryResultSize = ((ResultMessage.Rows)result).result.size();
-        lastMaxKey = getMaxKey(((ResultMessage.Rows)result).result);
-        return lastQueryResultSize != 0;
+        // find max
+        for (byte[] key : keys)
+            if (FBUtilities.compareUnsigned(cur, key) < 0)
+                cur = key;
+
+        // increment
+        for (int i = 0 ; i < cur.length ; i++)
+            if (++cur[i] != 0)
+                break;
+        return cur;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
index d593e57..6b1577c 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
@@ -21,126 +21,66 @@ package org.apache.cassandra.stress.operations;
  */
 
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.transport.SimpleClient;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.thrift.Compression;
-import org.apache.cassandra.thrift.CqlResult;
 import org.apache.cassandra.utils.UUIDGen;
 
-public class CqlInserter extends CQLOperation
+public class CqlInserter extends CqlOperation<Integer>
 {
-    private static List<ByteBuffer> values;
-    private static String cqlQuery = null;
 
-    public CqlInserter(Session client, int idx)
+    public CqlInserter(State state, long idx)
     {
-        super(client, idx);
+        super(state, idx);
     }
 
-    protected void run(CQLQueryExecutor executor) throws IOException
+    @Override
+    protected String buildQuery()
     {
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-            throw new RuntimeException("Super columns are not implemented for CQL");
+        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotesIfRequired(state.settings.schema.columnFamily));
 
-        if (values == null)
-            values = generateValues();
+        if (state.isCql2())
+            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
 
-        // Construct a query string once.
-        if (cqlQuery == null)
-        {
-            StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotesIfRequired("Standard1"));
-
-            if (session.cqlVersion.startsWith("2"))
-                query.append(" USING CONSISTENCY ").append(session.getConsistencyLevel().toString());
-
-            query.append(" SET ");
-
-            for (int i = 0; i < session.getColumnsPerKey(); i++)
-            {
-                if (i > 0)
-                    query.append(',');
-
-                if (session.timeUUIDComparator)
-                {
-                    if (session.cqlVersion.startsWith("3"))
-                        throw new UnsupportedOperationException("Cannot use UUIDs in column names with CQL3");
-
-                    query.append(wrapInQuotesIfRequired(UUIDGen.getTimeUUID().toString()))
-                         .append(" = ?");
-                }
-                else
-                {
-                    query.append(wrapInQuotesIfRequired("C" + i)).append(" = ?");
-                }
-            }
-
-            query.append(" WHERE KEY=?");
-            cqlQuery = query.toString();
-        }
+        query.append(" SET ");
 
-        List<String> queryParms = new ArrayList<String>();
-        for (int i = 0; i < session.getColumnsPerKey(); i++)
+        for (int i = 0 ; i < state.settings.columns.maxColumnsPerKey; i++)
         {
-            // Cell value
-            queryParms.add(getUnQuotedCqlBlob(values.get(i % values.size()).array(), session.cqlVersion.startsWith("3")));
-        }
-
-        String key = String.format("%0" + session.getTotalKeysLength() + "d", index);
-        queryParms.add(getUnQuotedCqlBlob(key, session.cqlVersion.startsWith("3")));
-
-        TimerContext context = session.latency.time();
-
-        boolean success = false;
-        String exceptionMessage = null;
-
-        for (int t = 0; t < session.getRetryTimes(); t++)
-        {
-            if (success)
-                break;
+            if (i > 0)
+                query.append(',');
 
-            try
+            if (state.settings.columns.useTimeUUIDComparator)
             {
-                success = executor.execute(cqlQuery, queryParms);
+                if (state.isCql3())
+                    throw new UnsupportedOperationException("Cannot use UUIDs in column names with CQL3");
+
+                query.append(wrapInQuotesIfRequired(UUIDGen.getTimeUUID().toString()))
+                        .append(" = ?");
             }
-            catch (Exception e)
+            else
             {
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
+                query.append(wrapInQuotesIfRequired("C" + i)).append(" = ?");
             }
         }
 
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error inserting key %s %s%n with query %s",
-                                index,
-                                session.getRetryTimes(),
-                                key,
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")",
-                                cqlQuery));
-        }
-
-        session.operations.getAndIncrement();
-        session.keys.getAndIncrement();
-        context.stop();
+        query.append(" WHERE KEY=?");
+        return query.toString();
     }
 
-    protected boolean validateThriftResult(CqlResult result)
+    @Override
+    protected List<ByteBuffer> getQueryParameters(byte[] key)
     {
-        return true;
+        final ArrayList<ByteBuffer> queryParams = new ArrayList<>();
+        final List<ByteBuffer> values = generateColumnValues();
+        queryParams.addAll(values);
+        queryParams.add(ByteBuffer.wrap(key));
+        return queryParams;
     }
 
-    protected boolean validateNativeResult(ResultMessage result)
+    @Override
+    protected CqlRunOp buildRunOp(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String keyid, ByteBuffer key)
     {
-        return true;
+        return new CqlRunOpAlwaysSucceed(client, query, queryId, params, keyid, key, 1);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CqlMultiGetter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlMultiGetter.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlMultiGetter.java
index ec645d4..80a7118 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlMultiGetter.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlMultiGetter.java
@@ -23,25 +23,20 @@ package org.apache.cassandra.stress.operations;
 
 import java.io.IOException;
 
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.transport.SimpleClient;
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.util.ThriftClient;
 
 public class CqlMultiGetter extends Operation
 {
-    public CqlMultiGetter(Session client, int idx)
-    {
-        super(client, idx);
-    }
-
-    public void run(CassandraClient client) throws IOException
+    public CqlMultiGetter(State state, long idx)
     {
+        super(state, idx);
         throw new RuntimeException("Multiget is not implemented for CQL");
     }
 
-    public void run(SimpleClient client) throws IOException
+    @Override
+    public void run(ThriftClient client) throws IOException
     {
-        throw new RuntimeException("Multiget is not implemented for CQL");
     }
+
 }


[2/6] Improve Stress Tool patch by Benedict; reviewed by Pavel Yaskevich for CASSANDRA-6199

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..6cef0bf
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsKey.java
@@ -0,0 +1,130 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+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;
+
+// Settings for key generation
+public class SettingsKey implements Serializable
+{
+
+    private final int keySize;
+    private final DistributionFactory distribution;
+    private final long[] range;
+
+    public SettingsKey(DistributionOptions options)
+    {
+        this.keySize = Integer.parseInt(options.size.value());
+        this.distribution = options.dist.get();
+        this.range = null;
+    }
+
+    public SettingsKey(PopulateOptions options)
+    {
+        this.keySize = Integer.parseInt(options.size.value());
+        this.distribution = null;
+        String[] bounds = options.populate.value().split("\\.\\.+");
+        this.range = new long[] { Long.parseLong(bounds[0]), Long.parseLong(bounds[1]) };
+    }
+
+    // Option Declarations
+
+    private static final class DistributionOptions extends GroupedOptions
+    {
+        final OptionDistribution dist;
+        final OptionSimple size = new OptionSimple("size=", "[0-9]+", "10", "Key size in bytes", false);
+
+        public DistributionOptions(String defaultLimit)
+        {
+            dist = new OptionDistribution("dist=", "GAUSSIAN(1.." + defaultLimit + ")");
+        }
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(dist, size);
+        }
+    }
+
+    private static final class PopulateOptions extends GroupedOptions
+    {
+        final OptionSimple populate;
+        final OptionSimple size = new OptionSimple("size=", "[0-9]+", "10", "Key size in bytes", false);
+
+        public PopulateOptions(String defaultLimit)
+        {
+            populate = new OptionSimple("populate=", "[0-9]+\\.\\.+[0-9]+",
+                    "1.." + defaultLimit,
+                    "Populate all keys in sequence", true);
+        }
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(populate, size);
+        }
+    }
+
+    public KeyGen newKeyGen()
+    {
+        if (range != null)
+            return new KeyGen(new DataGenHexFromOpIndex(range[0], range[1]), keySize);
+        return new KeyGen(new DataGenHexFromDistribution(distribution.get()), keySize);
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsKey get(Map<String, String[]> clArgs, SettingsCommand command)
+    {
+        // set default size to number of commands requested, unless set to err convergence, then use 1M
+        String defaultLimit = command.count <= 0 ? "1000000" : Long.toString(command.count);
+
+        String[] params = clArgs.remove("-key");
+        if (params == null)
+        {
+            // return defaults:
+            switch(command.type)
+            {
+                case WRITE:
+                case COUNTERWRITE:
+                    return new SettingsKey(new PopulateOptions(defaultLimit));
+                default:
+                    return new SettingsKey(new DistributionOptions(defaultLimit));
+            }
+        }
+        GroupedOptions options = GroupedOptions.select(params, new PopulateOptions(defaultLimit), new DistributionOptions(defaultLimit));
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -key options provided, see output for valid options");
+            System.exit(1);
+        }
+        return options instanceof PopulateOptions ?
+                new SettingsKey((PopulateOptions) options) :
+                new SettingsKey((DistributionOptions) options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-key", new PopulateOptions("N"), new DistributionOptions("N"));
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/SettingsLog.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsLog.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsLog.java
new file mode 100644
index 0000000..6a8e510
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsLog.java
@@ -0,0 +1,92 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.PrintStream;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class SettingsLog implements Serializable
+{
+
+    public final boolean noSummary;
+    public final File file;
+    public final int intervalMillis;
+
+    public SettingsLog(Options options)
+    {
+        noSummary = options.noSummmary.setByUser();
+
+        if (options.outputFile.setByUser())
+            file = new File(options.outputFile.value());
+        else
+            file = null;
+
+        String interval = options.interval.value();
+        if (interval.endsWith("ms"))
+            intervalMillis = Integer.parseInt(interval.substring(0, interval.length() - 2));
+        else if (interval.endsWith("s"))
+            intervalMillis = 1000 * Integer.parseInt(interval.substring(0, interval.length() - 1));
+        else
+            intervalMillis = 1000 * Integer.parseInt(interval);
+        if (intervalMillis <= 0)
+            throw new IllegalArgumentException("Log interval must be greater than zero");
+    }
+
+    public PrintStream getOutput() throws FileNotFoundException
+    {
+        return file == null ? new PrintStream(System.out) : new PrintStream(file);
+    }
+
+    // Option Declarations
+
+    public static final class Options extends GroupedOptions
+    {
+        final OptionSimple noSummmary = new OptionSimple("no-summary", "", null, "Disable printing of aggregate statistics at the end of a test", false);
+        final OptionSimple outputFile = new OptionSimple("file=", ".*", null, "Log to a file", false);
+        final OptionSimple interval = new OptionSimple("interval=", "[0-9]+(ms|s|)", "1s", "Log progress every <value> seconds or milliseconds", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(noSummmary, outputFile, interval);
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsLog get(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-log");
+        if (params == null)
+            return new SettingsLog(new Options());
+
+        GroupedOptions options = GroupedOptions.select(params, new Options());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -log options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsLog((Options) options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-log", new Options());
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..2092c02
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java
@@ -0,0 +1,200 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.PrintStream;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.stress.generatedata.Distribution;
+
+public class SettingsMisc implements Serializable
+{
+
+    static boolean maybeDoSpecial(Map<String, String[]> clArgs)
+    {
+        if (maybePrintHelp(clArgs))
+            return true;
+        if (maybePrintDistribution(clArgs))
+            return true;
+        return false;
+    }
+
+    static final class PrintDistribution extends GroupedOptions
+    {
+        final OptionDistribution dist = new OptionDistribution("dist=", null);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(dist);
+        }
+    }
+
+    static boolean maybePrintDistribution(Map<String, String[]> clArgs)
+    {
+        final String[] args = clArgs.get("print");
+        if (args == null)
+            return false;
+        final PrintDistribution dist = new PrintDistribution();
+        if (null == GroupedOptions.select(args, dist))
+        {
+            printHelpPrinter().run();
+            System.out.println("Invalid print options provided, see output for valid options");
+            System.exit(1);
+        }
+        printDistribution(dist.dist.get().get());
+        return true;
+    }
+
+    static void printDistribution(Distribution dist)
+    {
+        PrintStream out = System.out;
+        out.println("% of samples    Range       % of total");
+        String format = "%-16.1f%-12d%12.1f";
+        double rangemax = dist.inverseCumProb(1d) / 100d;
+        for (double d : new double[] { 0.1d, 0.2d, 0.3d, 0.4d, 0.5d, 0.6d, 0.7d, 0.8d, 0.9d, 0.95d, 0.99d, 1d })
+        {
+            double sampleperc = d * 100;
+            long max = dist.inverseCumProb(d);
+            double rangeperc = max/ rangemax;
+            out.println(String.format(format, sampleperc, max, rangeperc));
+        }
+    }
+
+    private static boolean maybePrintHelp(Map<String, String[]> clArgs)
+    {
+        if (!clArgs.containsKey("-?") && !clArgs.containsKey("help"))
+            return false;
+        String[] params = clArgs.remove("-?");
+        if (params == null)
+            params = clArgs.remove("help");
+        if (params.length == 0)
+        {
+            if (!clArgs.isEmpty())
+            {
+                if (clArgs.size() == 1)
+                {
+                    String p = clArgs.keySet().iterator().next();
+                    if (clArgs.get(p).length == 0)
+                        params = new String[] {p};
+                }
+            }
+            else
+            {
+                printHelp();
+                return true;
+            }
+        }
+        if (params.length == 1)
+        {
+            printHelp(params[0]);
+            return true;
+        }
+        throw new IllegalArgumentException("Invalid command/option provided to help");
+    }
+
+    public static void printHelp()
+    {
+        System.out.println("Usage: ./bin/cassandra-stress <command> [options]");
+        System.out.println();
+        System.out.println("---Commands---");
+        for (Command cmd : Command.values())
+        {
+            System.out.println(String.format("%-20s : %s", cmd.toString().toLowerCase(), cmd.description));
+        }
+        System.out.println();
+        System.out.println("---Options---");
+        for (CliOption cmd : CliOption.values())
+        {
+            System.out.println(String.format("-%-20s : %s", cmd.toString().toLowerCase(), cmd.description));
+        }
+    }
+
+    public static void printHelp(String command)
+    {
+        Command cmd = Command.get(command);
+        if (cmd != null)
+        {
+            cmd.printHelp();
+            return;
+        }
+        CliOption opt = CliOption.get(command);
+        if (opt != null)
+        {
+            opt.printHelp();
+            return;
+        }
+        printHelp();
+        throw new IllegalArgumentException("Invalid command or option provided to command help");
+    }
+
+    public static Runnable helpHelpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                System.out.println("Usage: ./bin/cassandra-stress help <command|option>");
+                System.out.println("Commands:");
+                for (Command cmd : Command.values())
+                    System.out.println("    " + cmd.toString().toLowerCase() + (cmd.extraName != null ? ", " + cmd.extraName : ""));
+                System.out.println("Options:");
+                for (CliOption op : CliOption.values())
+                    System.out.println("    -" + op.toString().toLowerCase() + (op.extraName != null ? ", " + op.extraName : ""));
+            }
+        };
+    }
+
+    public static Runnable printHelpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                GroupedOptions.printOptions(System.out, "print", new GroupedOptions()
+                {
+                    @Override
+                    public List<? extends Option> options()
+                    {
+                        return Arrays.asList(new OptionDistribution("dist=", null));
+                    }
+                });
+            }
+        };
+    }
+
+    public static Runnable sendToDaemonHelpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                System.out.println("Usage: -sendToDaemon <host>");
+                System.out.println();
+                System.out.println("Specify a host running the stress server to send this stress command to");
+            }
+        };
+    }
+
+    public static String getSendToDaemon(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-send-to");
+        if (params == null)
+            params = clArgs.remove("-sendto");
+        if (params == null)
+            return null;
+        if (params.length != 1)
+        {
+            sendToDaemonHelpPrinter().run();
+            System.out.println("Invalid -send-to specifier: " + Arrays.toString(params));
+            System.exit(1);
+        }
+        return params[0];
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..1800b28
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
@@ -0,0 +1,154 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import com.datastax.driver.core.ProtocolOptions;
+
+public class SettingsMode implements Serializable
+{
+
+    public final ConnectionAPI api;
+    public final ConnectionStyle style;
+    public final CqlVersion cqlVersion;
+    private final String compression;
+
+    public SettingsMode(GroupedOptions options)
+    {
+        if (options instanceof Cql3Options)
+        {
+            cqlVersion = CqlVersion.CQL3;
+            Cql3Options opts = (Cql3Options) options;
+            api = opts.useNative.setByUser() ? ConnectionAPI.JAVA_DRIVER_NATIVE : ConnectionAPI.THRIFT;
+            style = opts.usePrepared.setByUser() ? ConnectionStyle.CQL_PREPARED : ConnectionStyle.CQL;
+            compression = ProtocolOptions.Compression.valueOf(opts.useCompression.value().toUpperCase()).name();
+        }
+        else if (options instanceof Cql3SimpleNativeOptions)
+        {
+            cqlVersion = CqlVersion.CQL3;
+            Cql3SimpleNativeOptions opts = (Cql3SimpleNativeOptions) options;
+            api = ConnectionAPI.SIMPLE_NATIVE;
+            style = opts.usePrepared.setByUser() ? ConnectionStyle.CQL_PREPARED : ConnectionStyle.CQL;
+            compression = ProtocolOptions.Compression.NONE.name();
+        }
+        else if (options instanceof Cql2Options)
+        {
+            cqlVersion = CqlVersion.CQL2;
+            api = ConnectionAPI.THRIFT;
+            Cql2Options opts = (Cql2Options) options;
+            style = opts.usePrepared.setByUser() ? ConnectionStyle.CQL_PREPARED : ConnectionStyle.CQL;
+            compression = ProtocolOptions.Compression.NONE.name();
+        }
+        else if (options instanceof ThriftOptions)
+        {
+            ThriftOptions opts = (ThriftOptions) options;
+            cqlVersion = CqlVersion.NOCQL;
+            api = opts.smart.setByUser() ? ConnectionAPI.THRIFT_SMART : ConnectionAPI.THRIFT;
+            style = ConnectionStyle.THRIFT;
+            compression = ProtocolOptions.Compression.NONE.name();
+        }
+        else
+            throw new IllegalStateException();
+    }
+
+    public ProtocolOptions.Compression compression()
+    {
+        return ProtocolOptions.Compression.valueOf(compression);
+    }
+
+    // Option Declarations
+
+    private static final class Cql3Options extends GroupedOptions
+    {
+        final OptionSimple api = new OptionSimple("cql3", "", null, "", true);
+        final OptionSimple useNative = new OptionSimple("native", "", null, "", false);
+        final OptionSimple usePrepared = new OptionSimple("prepared", "", null, "", false);
+        final OptionSimple useCompression = new OptionSimple("compression=", "none|lz4|snappy", "none", "", false);
+        final OptionSimple port = new OptionSimple("port=", "[0-9]+", "9046", "", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(useNative, usePrepared, api, useCompression, port);
+        }
+    }
+
+    private static final class Cql3SimpleNativeOptions extends GroupedOptions
+    {
+        final OptionSimple api = new OptionSimple("cql3", "", null, "", true);
+        final OptionSimple useSimpleNative = new OptionSimple("simplenative", "", null, "", true);
+        final OptionSimple usePrepared = new OptionSimple("prepared", "", null, "", false);
+        final OptionSimple port = new OptionSimple("port=", "[0-9]+", "9046", "", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(useSimpleNative, usePrepared, api, port);
+        }
+    }
+
+    private static final class Cql2Options extends GroupedOptions
+    {
+        final OptionSimple api = new OptionSimple("cql2", "", null, "", true);
+        final OptionSimple usePrepared = new OptionSimple("prepared", "", null, "", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(usePrepared, api);
+        }
+    }
+
+    private static final class ThriftOptions extends GroupedOptions
+    {
+        final OptionSimple api = new OptionSimple("thrift", "", null, "", true);
+        final OptionSimple smart = new OptionSimple("smart", "", null, "", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(api, smart);
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsMode get(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-mode");
+        if (params == null)
+        {
+            ThriftOptions opts = new ThriftOptions();
+            opts.smart.accept("smart");
+            return new SettingsMode(opts);
+        }
+
+        GroupedOptions options = GroupedOptions.select(params, new ThriftOptions(), new Cql2Options(), new Cql3Options(), new Cql3SimpleNativeOptions());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -mode options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsMode(options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-mode", new ThriftOptions(), new Cql2Options(), new Cql3Options(), new Cql3SimpleNativeOptions());
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
new file mode 100644
index 0000000..2888987
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
@@ -0,0 +1,103 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class SettingsNode implements Serializable
+{
+
+    public final List<String> nodes;
+
+    public SettingsNode(Options options)
+    {
+        if (options.file.setByUser())
+        {
+            try
+            {
+                String node;
+                List<String> tmpNodes = new ArrayList<String>();
+                BufferedReader in = new BufferedReader(new InputStreamReader(new FileInputStream(options.file.value())));
+                try
+                {
+                    while ((node = in.readLine()) != null)
+                    {
+                        if (node.length() > 0)
+                            tmpNodes.add(node);
+                    }
+                    nodes = Arrays.asList(tmpNodes.toArray(new String[tmpNodes.size()]));
+                }
+                finally
+                {
+                    in.close();
+                }
+            }
+            catch(IOException ioe)
+            {
+                throw new RuntimeException(ioe);
+            }
+
+        }
+        else
+            nodes = Arrays.asList(options.list.value().split(","));
+    }
+
+    public String randomNode()
+    {
+        int index = (int) (Math.random() * nodes.size());
+        if (index >= nodes.size())
+            index = nodes.size() - 1;
+        return nodes.get(index);
+    }
+
+    // Option Declarations
+
+    public static final class Options extends GroupedOptions
+    {
+        final OptionSimple file = new OptionSimple("file=", ".*", null, "Node file (one per line)", false);
+        final OptionSimple list = new OptionSimple("", "[^=,]+(,[^=,]+)*", "localhost", "comma delimited list of hosts", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(file, list);
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsNode get(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-node");
+        if (params == null)
+            return new SettingsNode(new Options());
+
+        GroupedOptions options = GroupedOptions.select(params, new Options());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -node options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsNode((Options) options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-node", new Options());
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/SettingsPort.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsPort.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsPort.java
new file mode 100644
index 0000000..4d9b0ba
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsPort.java
@@ -0,0 +1,70 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class SettingsPort implements Serializable
+{
+
+    public final int nativePort;
+    public final int thriftPort;
+
+    public SettingsPort(PortOptions options)
+    {
+        nativePort = Integer.parseInt(options.nativePort.value());
+        thriftPort = Integer.parseInt(options.thriftPort.value());
+    }
+
+    // Option Declarations
+
+    private static final class PortOptions extends GroupedOptions
+    {
+        final OptionSimple nativePort = new OptionSimple("native=", "[0-9]+", "9042", "Use this port for the Cassandra native protocol", false);
+        final OptionSimple thriftPort = new OptionSimple("thrift=", "[0-9]+", "9160", "Use this port for the thrift protocol", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(nativePort, thriftPort);
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsPort get(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-port");
+        if (params == null)
+        {
+            return new SettingsPort(new PortOptions());
+        }
+        PortOptions options = GroupedOptions.select(params, new PortOptions());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -port options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsPort(options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-port", new PortOptions());
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/settings/SettingsRate.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsRate.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsRate.java
new file mode 100644
index 0000000..c5aff7a
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsRate.java
@@ -0,0 +1,116 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class SettingsRate implements Serializable
+{
+
+    public final boolean auto;
+    public final int minAutoThreads;
+    public final int maxAutoThreads;
+    public final int threadCount;
+    public final int opRateTargetPerSecond;
+
+    public SettingsRate(ThreadOptions options)
+    {
+        auto = false;
+        threadCount = Integer.parseInt(options.threads.value());
+        String rateOpt = options.rate.value();
+        opRateTargetPerSecond = Integer.parseInt(rateOpt.substring(0, rateOpt.length() - 2));
+        minAutoThreads = -1;
+        maxAutoThreads = -1;
+    }
+
+    public SettingsRate(AutoOptions auto)
+    {
+        this.auto = true;
+        this.minAutoThreads = Integer.parseInt(auto.minThreads.value());
+        this.maxAutoThreads = Integer.parseInt(auto.maxThreads.value());
+        this.threadCount = -1;
+        this.opRateTargetPerSecond = 0;
+    }
+
+
+    // Option Declarations
+
+    private static final class AutoOptions extends GroupedOptions
+    {
+        final OptionSimple auto = new OptionSimple("auto", "", null, "test with increasing number of threadCount until performance plateaus", false);
+        final OptionSimple minThreads = new OptionSimple("threads>=", "[0-9]+", "4", "run at least this many clients concurrently", false);
+        final OptionSimple maxThreads = new OptionSimple("threads<=", "[0-9]+", "1000", "run at most this many clients concurrently", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(auto, minThreads, maxThreads);
+        }
+    }
+
+    private static final class ThreadOptions extends GroupedOptions
+    {
+        final OptionSimple threads = new OptionSimple("threads=", "[0-9]+", null, "run this many clients concurrently", true);
+        final OptionSimple rate = new OptionSimple("limit=", "[0-9]+/s", "0/s", "limit operations per second across all clients", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(threads, rate);
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsRate get(Map<String, String[]> clArgs, SettingsCommand command)
+    {
+        String[] params = clArgs.remove("-rate");
+        if (params == null)
+        {
+            switch (command.type)
+            {
+                case WRITE:
+                case COUNTERWRITE:
+                    if (command.count > 0)
+                    {
+                        ThreadOptions options = new ThreadOptions();
+                        options.accept("threads=50");
+                        return new SettingsRate(options);
+                    }
+            }
+            return new SettingsRate(new AutoOptions());
+        }
+        GroupedOptions options = GroupedOptions.select(params, new AutoOptions(), new ThreadOptions());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -rate options provided, see output for valid options");
+            System.exit(1);
+        }
+        if (options instanceof AutoOptions)
+            return new SettingsRate((AutoOptions) options);
+        else if (options instanceof ThreadOptions)
+            return new SettingsRate((ThreadOptions) options);
+        else
+            throw new IllegalStateException();
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-rate", new ThreadOptions(), new AutoOptions());
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..6fc03e9
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
@@ -0,0 +1,236 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.thrift.*;
+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 boolean replicateOnWrite;
+    private final String compression;
+    private final String compactionStrategy;
+    public final String keyspace;
+    public final String columnFamily;
+
+    public SettingsSchema(Options options)
+    {
+        replicateOnWrite = !options.noReplicateOnWrite.setByUser();
+        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.compactionStrategy.value();
+        if (compactionStrategy != null)
+        {
+            try
+            {
+                CFMetaData.createCompactionStrategy(compactionStrategy);
+            } catch (ConfigurationException e)
+            {
+                throw new IllegalArgumentException("Invalid compaction strategy: " + compactionStrategy);
+            }
+        }
+        keyspace = options.keyspace.value();
+        columnFamily = options.columnFamily.value();
+    }
+
+    private void createKeyspacesCql3(StressSettings settings)
+    {
+//        settings.getJavaDriverClient().execute("create table Standard1")
+    }
+
+    public void createKeySpaces(StressSettings settings)
+    {
+        createKeySpacesThrift(settings);
+    }
+
+
+    /**
+     * Create Keyspace with Standard and Super/Counter column families
+     */
+    public void createKeySpacesThrift(StressSettings settings)
+    {
+        KsDef ksdef = new KsDef();
+
+        // column family for standard columns
+        CfDef standardCfDef = new CfDef(keyspace, columnFamily);
+        Map<String, String> compressionOptions = new HashMap<String, String>();
+        if (compression != null)
+            compressionOptions.put("sstable_compression", compression);
+
+        String comparator = settings.columns.comparator;
+        standardCfDef.setComparator_type(comparator)
+                .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);
+
+        // column family for standard counters
+        CfDef counterCfDef = new CfDef(keyspace, "Counter1")
+                .setComparator_type(comparator)
+                .setDefault_validation_class("CounterColumnType")
+                .setReplicate_on_write(replicateOnWrite)
+                .setCompression_options(compressionOptions);
+
+        // column family with counter super columns
+        CfDef counterSuperCfDef = new CfDef(keyspace, "SuperCounter1")
+                .setComparator_type(comparator)
+                .setDefault_validation_class("CounterColumnType")
+                .setReplicate_on_write(replicateOnWrite)
+                .setColumn_type("Super")
+                .setCompression_options(compressionOptions);
+
+        ksdef.setName(keyspace);
+        ksdef.setStrategy_class(replicationStrategy);
+
+        if (!replicationStrategyOptions.isEmpty())
+        {
+            ksdef.setStrategy_options(replicationStrategyOptions);
+        }
+
+        if (compactionStrategy != null)
+        {
+            standardCfDef.setCompaction_strategy(compactionStrategy);
+            superCfDef.setCompaction_strategy(compactionStrategy);
+            counterCfDef.setCompaction_strategy(compactionStrategy);
+            counterSuperCfDef.setCompaction_strategy(compactionStrategy);
+        }
+
+        ksdef.setCf_defs(new ArrayList<CfDef>(Arrays.asList(standardCfDef, superCfDef, counterCfDef, counterSuperCfDef)));
+
+        Cassandra.Client client = settings.getRawThriftClient(false);
+
+        try
+        {
+            client.system_add_keyspace(ksdef);
+
+            /* CQL3 counter cf */
+            client.set_cql_version("3.0.0"); // just to create counter cf for cql3
+
+            client.set_keyspace(keyspace);
+            client.execute_cql3_query(createCounterCFStatementForCQL3(settings), Compression.NONE, ConsistencyLevel.ONE);
+
+            if (settings.mode.cqlVersion.isCql())
+                client.set_cql_version(settings.mode.cqlVersion.connectVersion);
+            /* end */
+
+            System.out.println(String.format("Created keyspaces. Sleeping %ss for propagation.", settings.node.nodes.size()));
+            Thread.sleep(settings.node.nodes.size() * 1000); // seconds
+        }
+        catch (InvalidRequestException e)
+        {
+            System.err.println("Unable to create stress keyspace: " + e.getWhy());
+        }
+        catch (Exception e)
+        {
+            System.err.println("!!!! " + e.getMessage());
+        }
+    }
+
+    private ByteBuffer createCounterCFStatementForCQL3(StressSettings options)
+    {
+        StringBuilder counter3 = new StringBuilder("CREATE TABLE \"Counter3\" (KEY blob PRIMARY KEY, ");
+
+        for (int i = 0; i < options.columns.maxColumnsPerKey; i++)
+        {
+            counter3.append("c").append(i).append(" counter");
+            if (i != options.columns.maxColumnsPerKey - 1)
+                counter3.append(", ");
+        }
+        counter3.append(");");
+
+        return ByteBufferUtil.bytes(counter3.toString());
+    }
+
+    // Option Declarations
+
+    private static final class Options extends GroupedOptions
+    {
+        final OptionReplication replication = new OptionReplication();
+        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 columnFamily = new OptionSimple("columnfamily=", ".*", "Standard1", "The column family name to use", false);
+        final OptionSimple compactionStrategy = new OptionSimple("compaction=", ".*", null, "The compaction strategy to use", false);
+        final OptionSimple noReplicateOnWrite = new OptionSimple("no-replicate-on-write", "", null, "Set replicate_on_write to false for counters. Only counter add with CL=ONE will work", 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, columnFamily, compactionStrategy, noReplicateOnWrite, compression);
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsSchema get(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-schema");
+        if (params == null)
+            return new SettingsSchema(new Options());
+
+        GroupedOptions options = GroupedOptions.select(params, new Options());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -schema options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsSchema((Options) options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-schema", new Options());
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..3cb0402
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
@@ -0,0 +1,121 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.thrift.transport.TTransportFactory;
+
+public class SettingsTransport implements Serializable
+{
+
+    private final String fqFactoryClass;
+    private TTransportFactory factory;
+
+    public SettingsTransport(TOptions options)
+    {
+        if (options instanceof SSLOptions)
+        {
+            throw new UnsupportedOperationException();
+        }
+        else
+        {
+            this.fqFactoryClass = options.factory.value();
+            try
+            {
+                Class<?> clazz = Class.forName(fqFactoryClass);
+                if (!TTransportFactory.class.isAssignableFrom(clazz))
+                    throw new ClassCastException();
+                // check we can instantiate it
+                clazz.newInstance();
+            }
+            catch (Exception e)
+            {
+                throw new IllegalArgumentException("Invalid transport factory class: " + options.factory.value(), e);
+            }
+
+        }
+    }
+
+    public synchronized TTransportFactory getFactory()
+    {
+        if (factory == null)
+        {
+            try
+            {
+                this.factory = (TTransportFactory) Class.forName(fqFactoryClass).newInstance();
+            }
+            catch (Exception e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+        return factory;
+    }
+
+    // Option Declarations
+
+    static class TOptions extends GroupedOptions
+    {
+        final OptionSimple factory = new OptionSimple("factory=", ".*", "org.apache.cassandra.cli.transport.FramedTransportFactory", "Fully-qualified TTransportFactory class name for creating a connection. Note: For Thrift over SSL, use org.apache.cassandra.stress.SSLTransportFactory.", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(factory);
+        }
+    }
+
+    static final class SSLOptions extends TOptions
+    {
+        final OptionSimple trustStore = new OptionSimple("truststore=", ".*", null, "SSL: full path to truststore", false);
+        final OptionSimple trustStorePw = new OptionSimple("truststore-password=", ".*", null, "", false);
+        final OptionSimple protocol = new OptionSimple("ssl-protocol=", ".*", "TLS", "SSL: connections protocol to use", false);
+        final OptionSimple alg = new OptionSimple("ssl-alg=", ".*", "SunX509", "SSL: algorithm", false);
+        final OptionSimple storeType = new OptionSimple("store-type=", ".*", "TLS", "SSL: comma delimited list of encryption suites to use", false);
+        final OptionSimple ciphers = new OptionSimple("ssl-ciphers=", ".*", "TLS", "SSL: comma delimited list of encryption suites to use", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(factory, trustStore, trustStorePw, protocol, alg, storeType, ciphers);
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsTransport get(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-transport");
+        if (params == null)
+            return new SettingsTransport(new TOptions());
+
+        GroupedOptions options = GroupedOptions.select(params, new TOptions());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -transport options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsTransport((TOptions) options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-transport", new TOptions());
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..ec4db96
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
@@ -0,0 +1,239 @@
+package org.apache.cassandra.stress.settings;
+
+import java.io.Serializable;
+import java.util.*;
+
+import com.datastax.driver.core.Metadata;
+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.thrift.Cassandra;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.thrift.TFramedTransportFactory;
+import org.apache.cassandra.transport.SimpleClient;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+
+public class StressSettings implements Serializable
+{
+
+    public final SettingsCommand command;
+    public final SettingsRate rate;
+    public final SettingsKey keys;
+    public final SettingsColumn columns;
+    public final SettingsLog log;
+    public final SettingsMode mode;
+    public final SettingsNode node;
+    public final SettingsSchema schema;
+    public final SettingsTransport transport;
+    public final SettingsPort port;
+    public final String sendToDaemon;
+
+    public StressSettings(SettingsCommand command, SettingsRate rate, SettingsKey keys, SettingsColumn columns, SettingsLog log, SettingsMode mode, SettingsNode node, SettingsSchema schema, SettingsTransport transport, SettingsPort port, String sendToDaemon)
+    {
+        this.command = command;
+        this.rate = rate;
+        this.keys = keys;
+        this.columns = columns;
+        this.log = log;
+        this.mode = mode;
+        this.node = node;
+        this.schema = schema;
+        this.transport = transport;
+        this.port = port;
+        this.sendToDaemon = sendToDaemon;
+    }
+
+    public SmartThriftClient getSmartThriftClient()
+    {
+        Metadata metadata = getJavaDriverClient().getCluster().getMetadata();
+        return new SmartThriftClient(this, schema.keyspace, metadata);
+    }
+
+    /**
+     * Thrift client connection
+     * @return cassandra client connection
+     */
+    public SimpleThriftClient getThriftClient()
+    {
+        return new SimpleThriftClient(getRawThriftClient(node.randomNode(), true));
+    }
+
+    public Cassandra.Client getRawThriftClient(boolean setKeyspace)
+    {
+        return getRawThriftClient(node.randomNode(), setKeyspace);
+    }
+
+    public Cassandra.Client getRawThriftClient(String host)
+    {
+        return getRawThriftClient(host, true);
+    }
+
+    public Cassandra.Client getRawThriftClient(String host, boolean setKeyspace)
+    {
+        TSocket socket = new TSocket(host, port.thriftPort);
+        Cassandra.Client client;
+
+        try
+        {
+            TTransport transport = this.transport.getFactory().getTransport(socket);
+            transport.open();
+
+            client = new Cassandra.Client(new TBinaryProtocol(transport));
+
+            if (mode.cqlVersion.isCql())
+                client.set_cql_version(mode.cqlVersion.connectVersion);
+
+            if (setKeyspace)
+                client.set_keyspace("Keyspace1");
+        }
+        catch (InvalidRequestException e)
+        {
+            throw new RuntimeException(e.getWhy());
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e.getMessage());
+        }
+
+        return client;
+    }
+
+
+    public SimpleClient getSimpleNativeClient()
+    {
+        try
+        {
+            String currentNode = node.randomNode();
+            SimpleClient client = new SimpleClient(currentNode, port.nativePort);
+            client.connect(false);
+            client.execute("USE \"Keyspace1\";", org.apache.cassandra.db.ConsistencyLevel.ONE);
+            return client;
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e.getMessage());
+        }
+    }
+
+    private static volatile JavaDriverClient client;
+
+    public JavaDriverClient getJavaDriverClient()
+    {
+        if (client != null)
+            return client;
+
+        try
+        {
+            synchronized (this)
+            {
+                String currentNode = node.randomNode();
+                if (client != null)
+                    return client;
+
+                JavaDriverClient c = new JavaDriverClient(currentNode, port.nativePort);
+                c.connect(mode.compression());
+                c.execute("USE \"Keyspace1\";", org.apache.cassandra.db.ConsistencyLevel.ONE);
+                return client = c;
+            }
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void maybeCreateKeyspaces()
+    {
+        if (command.type == Command.WRITE || command.type == Command.COUNTERWRITE)
+            schema.createKeySpaces(this);
+
+    }
+
+    public static StressSettings parse(String[] args)
+    {
+        final Map<String, String[]> clArgs = parseMap(args);
+        if (clArgs.containsKey("legacy"))
+            return Legacy.build(Arrays.copyOfRange(args, 1, args.length));
+        if (SettingsMisc.maybeDoSpecial(clArgs))
+            System.exit(1);
+        return get(clArgs);
+    }
+
+    public static StressSettings get(Map<String, String[]> clArgs)
+    {
+        SettingsCommand command = SettingsCommand.get(clArgs);
+        if (command == null)
+            throw new IllegalArgumentException("No command specified");
+        String sendToDaemon = SettingsMisc.getSendToDaemon(clArgs);
+        SettingsPort port = SettingsPort.get(clArgs);
+        SettingsRate rate = SettingsRate.get(clArgs, command);
+        SettingsKey keys = SettingsKey.get(clArgs, command);
+        SettingsColumn columns = SettingsColumn.get(clArgs);
+        SettingsLog log = SettingsLog.get(clArgs);
+        SettingsMode mode = SettingsMode.get(clArgs);
+        SettingsNode node = SettingsNode.get(clArgs);
+        SettingsSchema schema = SettingsSchema.get(clArgs);
+        SettingsTransport transport = SettingsTransport.get(clArgs);
+        if (!clArgs.isEmpty())
+        {
+            printHelp();
+            System.out.println("Error processing command line arguments. The following were ignored:");
+            for (Map.Entry<String, String[]> e : clArgs.entrySet())
+            {
+                System.out.print(e.getKey());
+                for (String v : e.getValue())
+                {
+                    System.out.print(" ");
+                    System.out.print(v);
+                }
+                System.out.println();
+            }
+            System.exit(1);
+        }
+        return new StressSettings(command, rate, keys, columns, log, mode, node, schema, transport, port, sendToDaemon);
+    }
+
+    private static Map<String, String[]> parseMap(String[] args)
+    {
+        // first is the main command/operation, so specified without a -
+        if (args.length == 0)
+        {
+            System.out.println("No command provided");
+            printHelp();
+            System.exit(1);
+        }
+        final LinkedHashMap<String, String[]> r = new LinkedHashMap<>();
+        String key = null;
+        List<String> params = new ArrayList<>();
+        for (int i = 0 ; i < args.length ; i++)
+        {
+            if (i == 0 || args[i].startsWith("-"))
+            {
+                if (i > 0)
+                    r.put(key, params.toArray(new String[0]));
+                key = args[i].toLowerCase();
+                params.clear();
+            }
+            else
+                params.add(args[i]);
+        }
+        r.put(key, params.toArray(new String[0]));
+        return r;
+    }
+
+    public static void printHelp()
+    {
+        SettingsMisc.printHelp();
+    }
+
+    public synchronized void disconnect()
+    {
+        if (client == null)
+            return;
+
+        client.disconnect();
+        client = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/CassandraClient.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/CassandraClient.java b/tools/stress/src/org/apache/cassandra/stress/util/CassandraClient.java
deleted file mode 100644
index 5136a55..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/util/CassandraClient.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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.util;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.cassandra.thrift.Cassandra.Client;
-import org.apache.thrift.protocol.TProtocol;
-
-public class CassandraClient extends Client
-{
-    public Map<Integer, Integer> preparedStatements = new HashMap<Integer, Integer>();
-
-    public CassandraClient(TProtocol protocol)
-    {
-        super(protocol);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/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
new file mode 100644
index 0000000..f13c1b6
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
@@ -0,0 +1,148 @@
+/**
+ * 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.util;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import javax.net.ssl.SSLContext;
+
+import com.datastax.driver.core.*;
+import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.security.SSLFactory;
+import org.apache.cassandra.utils.FBUtilities;
+import org.jboss.netty.logging.InternalLoggerFactory;
+import org.jboss.netty.logging.Slf4JLoggerFactory;
+
+public class JavaDriverClient
+{
+
+    static
+    {
+        InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
+    }
+
+    public final String host;
+    public final int port;
+    private final EncryptionOptions.ClientEncryptionOptions encryptionOptions;
+    private Cluster cluster;
+    private Session session;
+
+    public JavaDriverClient(String host, int port)
+    {
+        this(host, port, new EncryptionOptions.ClientEncryptionOptions());
+    }
+
+    public JavaDriverClient(String host, int port, EncryptionOptions.ClientEncryptionOptions encryptionOptions)
+    {
+        this.host = host;
+        this.port = port;
+        this.encryptionOptions = encryptionOptions;
+    }
+
+    public PreparedStatement prepare(String query)
+    {
+        return getSession().prepare(query);
+    }
+
+    public void connect(ProtocolOptions.Compression compression) throws Exception
+    {
+        Cluster.Builder clusterBuilder = Cluster.builder()
+                .addContactPoint(host).withPort(port);
+        clusterBuilder.withCompression(compression);
+        if (encryptionOptions.enabled)
+        {
+            SSLContext sslContext;
+            sslContext = SSLFactory.createSSLContext(encryptionOptions, true);
+            SSLOptions sslOptions = new SSLOptions(sslContext, encryptionOptions.cipher_suites);
+            clusterBuilder.withSSL(sslOptions);
+        }
+        cluster = clusterBuilder.build();
+        Metadata metadata = cluster.getMetadata();
+        System.out.printf("Connected to cluster: %s\n",
+                metadata.getClusterName());
+        for (Host host : metadata.getAllHosts())
+        {
+            System.out.printf("Datatacenter: %s; Host: %s; Rack: %s\n",
+                    host.getDatacenter(), host.getAddress(), host.getRack());
+        }
+
+        session = cluster.connect();
+    }
+
+    public Cluster getCluster()
+    {
+        return cluster;
+    }
+
+    public Session getSession()
+    {
+        return session;
+    }
+
+    public ResultSet execute(String query, org.apache.cassandra.db.ConsistencyLevel consistency)
+    {
+        SimpleStatement stmt = new SimpleStatement(query);
+        stmt.setConsistencyLevel(from(consistency));
+        return getSession().execute(stmt);
+    }
+
+    public ResultSet executePrepared(PreparedStatement stmt, List<ByteBuffer> queryParams, org.apache.cassandra.db.ConsistencyLevel consistency)
+    {
+
+        stmt.setConsistencyLevel(from(consistency));
+        BoundStatement bstmt = stmt.bind(queryParams.toArray(new ByteBuffer[queryParams.size()]));
+        return getSession().execute(bstmt);
+    }
+
+    /**
+     * Get ConsistencyLevel from a C* ConsistencyLevel. This exists in the Java Driver ConsistencyLevel,
+     * but it is not public.
+     *
+     * @param cl
+     * @return
+     */
+    ConsistencyLevel from(org.apache.cassandra.db.ConsistencyLevel cl)
+    {
+        switch (cl)
+        {
+            case ANY:
+                return com.datastax.driver.core.ConsistencyLevel.ANY;
+            case ONE:
+                return com.datastax.driver.core.ConsistencyLevel.ONE;
+            case TWO:
+                return com.datastax.driver.core.ConsistencyLevel.TWO;
+            case THREE:
+                return com.datastax.driver.core.ConsistencyLevel.THREE;
+            case QUORUM:
+                return com.datastax.driver.core.ConsistencyLevel.QUORUM;
+            case ALL:
+                return com.datastax.driver.core.ConsistencyLevel.ALL;
+            case LOCAL_QUORUM:
+                return com.datastax.driver.core.ConsistencyLevel.LOCAL_QUORUM;
+            case EACH_QUORUM:
+                return com.datastax.driver.core.ConsistencyLevel.EACH_QUORUM;
+        }
+        throw new AssertionError();
+    }
+
+    public void disconnect()
+    {
+        FBUtilities.waitOnFuture(cluster.shutdown());
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/Operation.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/Operation.java b/tools/stress/src/org/apache/cassandra/stress/util/Operation.java
deleted file mode 100644
index e2e12f8..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/util/Operation.java
+++ /dev/null
@@ -1,334 +0,0 @@
-/**
- * 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.util;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.IOException;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.Map;
-import java.util.HashMap;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-
-import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.Stress;
-import org.apache.cassandra.transport.SimpleClient;
-import org.apache.cassandra.thrift.Compression;
-import org.apache.cassandra.thrift.CqlPreparedResult;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Hex;
-import org.apache.cassandra.utils.UUIDGen;
-
-public abstract class Operation
-{
-    public final int index;
-
-    protected final Session session;
-    protected static volatile Double nextGaussian = null;
-
-    public Operation(int idx)
-    {
-        index = idx;
-        session = Stress.session;
-    }
-
-    public Operation(Session client, int idx)
-    {
-        index = idx;
-        session = client;
-    }
-
-    /**
-     * Run operation
-     * @param client Cassandra Thrift client connection
-     * @throws IOException on any I/O error.
-     */
-    public abstract void run(CassandraClient client) throws IOException;
-
-    public void run(SimpleClient client) throws IOException {}
-
-    // Utility methods
-
-    protected List<ByteBuffer> generateValues()
-    {
-        if (session.averageSizeValues)
-        {
-            return generateRandomizedValues();
-        }
-
-        List<ByteBuffer> values = new ArrayList<ByteBuffer>();
-
-        for (int i = 0; i < session.getCardinality(); i++)
-        {
-            String hash = getMD5(Integer.toString(i));
-            int times = session.getColumnSize() / hash.length();
-            int sumReminder = session.getColumnSize() % hash.length();
-
-            String value = multiplyString(hash, times) + hash.substring(0, sumReminder);
-            values.add(ByteBuffer.wrap(value.getBytes()));
-        }
-
-        return values;
-    }
-
-    /**
-     * Generate values of average size specified by -S, up to cardinality specified by -C
-     * @return Collection of the values
-     */
-    protected List<ByteBuffer> generateRandomizedValues()
-    {
-        List<ByteBuffer> values = new ArrayList<ByteBuffer>();
-
-        int limit = 2 * session.getColumnSize();
-
-        for (int i = 0; i < session.getCardinality(); i++)
-        {
-            byte[] value = new byte[Stress.randomizer.nextInt(limit)];
-            Stress.randomizer.nextBytes(value);
-            values.add(ByteBuffer.wrap(value));
-        }
-
-        return values;
-    }
-
-    /**
-     * key generator using Gauss or Random algorithm
-     * @return byte[] representation of the key string
-     */
-    protected byte[] generateKey()
-    {
-        return (session.useRandomGenerator()) ? generateRandomKey() : generateGaussKey();
-    }
-
-    /**
-     * Random key generator
-     * @return byte[] representation of the key string
-     */
-    private byte[] generateRandomKey()
-    {
-        String format = "%0" + session.getTotalKeysLength() + "d";
-        return String.format(format, Stress.randomizer.nextInt(Stress.session.getNumDifferentKeys() - 1)).getBytes(UTF_8);
-    }
-
-    /**
-     * Gauss key generator
-     * @return byte[] representation of the key string
-     */
-    private byte[] generateGaussKey()
-    {
-        String format = "%0" + session.getTotalKeysLength() + "d";
-
-        for (;;)
-        {
-            double token = nextGaussian(session.getMean(), session.getSigma());
-
-            if (0 <= token && token < session.getNumDifferentKeys())
-            {
-                return String.format(format, (int) token).getBytes(UTF_8);
-            }
-        }
-    }
-
-    /**
-     * Gaussian distribution.
-     * @param mu is the mean
-     * @param sigma is the standard deviation
-     *
-     * @return next Gaussian distribution number
-     */
-    private static double nextGaussian(int mu, float sigma)
-    {
-        Random random = Stress.randomizer;
-
-        Double currentState = nextGaussian;
-        nextGaussian = null;
-
-        if (currentState == null)
-        {
-            double x2pi  = random.nextDouble() * 2 * Math.PI;
-            double g2rad = Math.sqrt(-2.0 * Math.log(1.0 - random.nextDouble()));
-
-            currentState = Math.cos(x2pi) * g2rad;
-            nextGaussian = Math.sin(x2pi) * g2rad;
-        }
-
-        return mu + currentState * sigma;
-    }
-
-    /**
-     * MD5 string generation
-     * @param input String
-     * @return md5 representation of the string
-     */
-    private String getMD5(String input)
-    {
-        MessageDigest md = FBUtilities.threadLocalMD5Digest();
-        byte[] messageDigest = md.digest(input.getBytes(UTF_8));
-        StringBuilder hash = new StringBuilder(new BigInteger(1, messageDigest).toString(16));
-
-        while (hash.length() < 32)
-            hash.append("0").append(hash);
-
-        return hash.toString();
-    }
-
-    /**
-     * Equal to python/ruby - 's' * times
-     * @param str String to multiple
-     * @param times multiplication times
-     * @return multiplied string
-     */
-    private String multiplyString(String str, int times)
-    {
-        StringBuilder result = new StringBuilder();
-
-        for (int i = 0; i < times; i++)
-            result.append(str);
-
-        return result.toString();
-    }
-
-    protected ByteBuffer columnName(int index, boolean timeUUIDComparator)
-    {
-        return timeUUIDComparator
-                ? TimeUUIDType.instance.decompose(UUIDGen.getTimeUUID())
-                : ByteBufferUtil.bytes(String.format("C%d", index));
-    }
-
-    protected String getExceptionMessage(Exception e)
-    {
-        String className = e.getClass().getSimpleName();
-        String message = (e instanceof InvalidRequestException) ? ((InvalidRequestException) e).getWhy() : e.getMessage();
-        return (message == null) ? "(" + className + ")" : String.format("(%s): %s", className, message);
-    }
-
-    protected void error(String message) throws IOException
-    {
-        if (!session.ignoreErrors())
-            throw new IOException(message);
-        else
-            System.err.println(message);
-    }
-
-    protected String getUnQuotedCqlBlob(String term, boolean isCQL3)
-    {
-        return getUnQuotedCqlBlob(term.getBytes(), isCQL3);
-    }
-
-    protected String getUnQuotedCqlBlob(byte[] term, boolean isCQL3)
-    {
-        return isCQL3
-             ? "0x" + Hex.bytesToHex(term)
-             : Hex.bytesToHex(term);
-    }
-
-    protected List<ByteBuffer> queryParamsAsByteBuffer(List<String> queryParams)
-    {
-        return Lists.transform(queryParams, new Function<String, ByteBuffer>()
-        {
-            public ByteBuffer apply(String param)
-            {
-                if (param.startsWith("0x"))
-                    param = param.substring(2);
-                return ByteBufferUtil.hexToBytes(param);
-            }
-        });
-    }
-
-    /**
-     * Constructs a CQL query string by replacing instances of the character
-     * '?', with the corresponding parameter.
-     *
-     * @param query base query string to format
-     * @param parms sequence of string query parameters
-     * @return formatted CQL query string
-     */
-    protected static String formatCqlQuery(String query, List<String> parms)
-    {
-        int marker, position = 0;
-        StringBuilder result = new StringBuilder();
-
-        if (-1 == (marker = query.indexOf('?')) || parms.size() == 0)
-            return query;
-
-        for (String parm : parms)
-        {
-            result.append(query.substring(position, marker));
-            result.append(parm);
-
-            position = marker + 1;
-            if (-1 == (marker = query.indexOf('?', position + 1)))
-                break;
-        }
-
-        if (position < query.length())
-            result.append(query.substring(position));
-
-        return result.toString();
-    }
-
-    protected Integer getPreparedStatement(CassandraClient client, String cqlQuery) throws Exception
-    {
-        Integer statementId = client.preparedStatements.get(cqlQuery.hashCode());
-        if (statementId == null)
-        {
-            CqlPreparedResult response = session.cqlVersion.startsWith("3")
-                                       ? client.prepare_cql3_query(ByteBufferUtil.bytes(cqlQuery), Compression.NONE)
-                                       : client.prepare_cql_query(ByteBufferUtil.bytes(cqlQuery), Compression.NONE);
-            statementId = response.itemId;
-            client.preparedStatements.put(cqlQuery.hashCode(), statementId);
-        }
-
-        return statementId;
-    }
-
-    private static final Map<Integer, byte[]> preparedStatementsNative = new HashMap<Integer, byte[]>();
-
-    protected static byte[] getPreparedStatement(SimpleClient client, String cqlQuery) throws Exception
-    {
-        byte[] statementId = preparedStatementsNative.get(cqlQuery.hashCode());
-        if (statementId == null)
-        {
-            statementId = client.prepare(cqlQuery).statementId.bytes;
-            preparedStatementsNative.put(cqlQuery.hashCode(), statementId);
-        }
-        return statementId;
-    }
-
-    protected String wrapInQuotesIfRequired(String string)
-    {
-        return session.cqlVersion.startsWith("3")
-                ? "\"" + string + "\""
-                : string;
-    }
-
-    public interface CQLQueryExecutor
-    {
-        public boolean execute(String query, List<String> queryParameters) throws Exception;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java b/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java
new file mode 100644
index 0000000..af1381b
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java
@@ -0,0 +1,107 @@
+package org.apache.cassandra.stress.util;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+// represents a sample of long (latencies) together with the probability of selection of each sample (i.e. the ratio of
+// samples to total number of events). This is used to ensure that, when merging, the result has samples from each
+// with equal probability
+public final class SampleOfLongs
+{
+
+    // nanos
+    final long[] sample;
+
+    // probability with which each sample was selected
+    final double p;
+
+    SampleOfLongs(long[] sample, int p)
+    {
+        this.sample = sample;
+        this.p = 1 / (float) p;
+    }
+
+    SampleOfLongs(long[] sample, double p)
+    {
+        this.sample = sample;
+        this.p = p;
+    }
+
+    static SampleOfLongs merge(Random rnd, List<SampleOfLongs> merge, int maxSamples)
+    {
+        int maxLength = 0;
+        double targetp = 1;
+        for (SampleOfLongs sampleOfLongs : merge)
+        {
+            maxLength += sampleOfLongs.sample.length;
+            targetp = Math.min(targetp, sampleOfLongs.p);
+        }
+        long[] sample = new long[maxLength];
+        int count = 0;
+        for (SampleOfLongs latencies : merge)
+        {
+            long[] in = latencies.sample;
+            double p = targetp / latencies.p;
+            for (int i = 0 ; i < in.length ; i++)
+                if (rnd.nextDouble() < p)
+                    sample[count++] = in[i];
+        }
+        if (count > maxSamples)
+        {
+            targetp = subsample(rnd, maxSamples, sample, count, targetp);
+            count = maxSamples;
+        }
+        sample = Arrays.copyOf(sample, count);
+        Arrays.sort(sample);
+        return new SampleOfLongs(sample, targetp);
+    }
+
+    public SampleOfLongs subsample(Random rnd, int maxSamples)
+    {
+        if (maxSamples > sample.length)
+            return this;
+
+        long[] sample = this.sample.clone();
+        double p = subsample(rnd, maxSamples, sample, sample.length, this.p);
+        sample = Arrays.copyOf(sample, maxSamples);
+        return new SampleOfLongs(sample, p);
+    }
+
+    private static double subsample(Random rnd, int maxSamples, long[] sample, int count, double p)
+    {
+        // want exactly maxSamples, so select random indexes up to maxSamples
+        for (int i = 0 ; i < maxSamples ; i++)
+        {
+            int take = i + rnd.nextInt(count - i);
+            long tmp = sample[i];
+            sample[i] = sample[take];
+            sample[take] = tmp;
+        }
+
+        // calculate new p; have selected with probability maxSamples / count
+        // so multiply p by this probability
+        p *= maxSamples / (double) sample.length;
+        return p;
+    }
+
+    public double medianLatency()
+    {
+        if (sample.length == 0)
+            return 0;
+        return sample[sample.length >> 1] * 0.000001d;
+    }
+
+    // 0 < rank < 1
+    public double rankLatency(float rank)
+    {
+        if (sample.length == 0)
+            return 0;
+        int index = (int)(rank * sample.length);
+        if (index >= sample.length)
+            index = sample.length - 1;
+        return sample[index] * 0.000001d;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/util/SimpleThriftClient.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/SimpleThriftClient.java b/tools/stress/src/org/apache/cassandra/stress/util/SimpleThriftClient.java
new file mode 100644
index 0000000..9e8b046
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/util/SimpleThriftClient.java
@@ -0,0 +1,90 @@
+package org.apache.cassandra.stress.util;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
+
+public class SimpleThriftClient implements ThriftClient
+{
+
+    final Cassandra.Client client;
+    public SimpleThriftClient(Cassandra.Client client)
+    {
+        this.client = client;
+    }
+
+    public void batch_mutate(Map<ByteBuffer, Map<String, List<Mutation>>> record, ConsistencyLevel consistencyLevel) throws TException
+    {
+        client.batch_mutate(record, consistencyLevel);
+    }
+
+    @Override
+    public List<ColumnOrSuperColumn> get_slice(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        return client.get_slice(key, column_parent, predicate, consistency_level);
+    }
+
+    @Override
+    public List<KeySlice> get_indexed_slices(ColumnParent column_parent, IndexClause index_clause, SlicePredicate column_predicate, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        return client.get_indexed_slices(column_parent, index_clause, column_predicate, consistency_level);
+    }
+
+    @Override
+    public List<KeySlice> get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        return client.get_range_slices(column_parent, predicate, range, consistency_level);
+    }
+
+    @Override
+    public Map<ByteBuffer, List<ColumnOrSuperColumn>> multiget_slice(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        return client.multiget_slice(keys, column_parent, predicate, consistency_level);
+    }
+
+    @Override
+    public void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    {
+        client.insert(key, column_parent, column, consistency_level);
+    }
+
+    @Override
+    public Integer prepare_cql3_query(String query, Compression compression) throws InvalidRequestException, TException
+    {
+        return client.prepare_cql3_query(ByteBufferUtil.bytes(query), compression).itemId;
+    }
+
+    @Override
+    public CqlResult execute_prepared_cql_query(int itemId, ByteBuffer key, List<ByteBuffer> values) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    {
+        return client.execute_prepared_cql_query(itemId, values);
+    }
+
+    @Override
+    public Integer prepare_cql_query(String query, Compression compression) throws InvalidRequestException, TException
+    {
+        return client.prepare_cql_query(ByteBufferUtil.bytes(query), compression).itemId;
+    }
+
+    @Override
+    public CqlResult execute_cql3_query(String query, ByteBuffer key, Compression compression, ConsistencyLevel consistency) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    {
+        return client.execute_cql3_query(ByteBufferUtil.bytes(query), compression, consistency);
+    }
+
+    @Override
+    public CqlResult execute_prepared_cql3_query(int itemId, ByteBuffer key, List<ByteBuffer> values, ConsistencyLevel consistency) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    {
+        return client.execute_prepared_cql3_query(itemId, values, consistency);
+    }
+
+    @Override
+    public CqlResult execute_cql_query(String query, ByteBuffer key, Compression compression) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    {
+        return client.execute_cql_query(ByteBufferUtil.bytes(query), compression);
+    }
+}


[4/6] Improve Stress Tool patch by Benedict; reviewed by Pavel Yaskevich for CASSANDRA-6199

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java
new file mode 100644
index 0000000..1f734be
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java
@@ -0,0 +1,566 @@
+/*
+ * 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.operations;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.google.common.base.Function;
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.settings.ConnectionStyle;
+import org.apache.cassandra.stress.settings.CqlVersion;
+import org.apache.cassandra.stress.util.JavaDriverClient;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.thrift.Compression;
+import org.apache.cassandra.thrift.CqlResult;
+import org.apache.cassandra.thrift.ThriftConversion;
+import org.apache.cassandra.transport.SimpleClient;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
+
+public abstract class CqlOperation<V> extends Operation
+{
+
+    protected abstract List<ByteBuffer> getQueryParameters(byte[] key);
+    protected abstract String buildQuery();
+    protected abstract CqlRunOp<V> buildRunOp(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String id, ByteBuffer key);
+
+    public CqlOperation(State state, long idx)
+    {
+        super(state, idx);
+        if (state.settings.columns.useSuperColumns)
+            throw new IllegalStateException("Super columns are not implemented for CQL");
+        if (state.settings.columns.variableColumnCount)
+            throw new IllegalStateException("Variable column counts are not implemented for CQL");
+    }
+
+    protected CqlRunOp<V> run(final ClientWrapper client, final List<ByteBuffer> queryParams, final ByteBuffer key, final String keyid) throws IOException
+    {
+        final CqlRunOp<V> op;
+        if (state.settings.mode.style == ConnectionStyle.CQL_PREPARED)
+        {
+            final Object id;
+            Object idobj = state.getCqlCache();
+            if (idobj == null)
+            {
+                try
+                {
+                    id = client.createPreparedStatement(buildQuery());
+                } catch (TException e)
+                {
+                    throw new RuntimeException(e);
+                }
+                state.storeCqlCache(id);
+            }
+            else
+                id = idobj;
+
+            op = buildRunOp(client, null, id, queryParams, keyid, key);
+        }
+        else
+        {
+            final String query;
+            Object qobj = state.getCqlCache();
+            if (qobj == null)
+                state.storeCqlCache(query = buildQuery());
+            else
+                query = qobj.toString();
+
+            op = buildRunOp(client, query, null, queryParams, keyid, key);
+        }
+
+        timeWithRetry(op);
+        return op;
+    }
+
+    protected void run(final ClientWrapper client) throws IOException
+    {
+        final byte[] key = getKey().array();
+        final List<ByteBuffer> queryParams = getQueryParameters(key);
+        run(client, queryParams, ByteBuffer.wrap(key), new String(key));
+    }
+
+    // Classes to process Cql results
+
+    // Always succeeds so long as the query executes without error; provides a keyCount to increment on instantiation
+    protected final class CqlRunOpAlwaysSucceed extends CqlRunOp<Integer>
+    {
+
+        final int keyCount;
+
+        protected CqlRunOpAlwaysSucceed(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String id, ByteBuffer key, int keyCount)
+        {
+            super(client, query, queryId, RowCountHandler.INSTANCE, params, id, key);
+            this.keyCount = keyCount;
+        }
+
+        @Override
+        public boolean validate(Integer result)
+        {
+            return true;
+        }
+
+        @Override
+        public int keyCount()
+        {
+            return keyCount;
+        }
+    }
+
+    // Succeeds so long as the result set is nonempty, and the query executes without error
+    protected final class CqlRunOpTestNonEmpty extends CqlRunOp<Integer>
+    {
+
+        protected CqlRunOpTestNonEmpty(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String id, ByteBuffer key)
+        {
+            super(client, query, queryId, RowCountHandler.INSTANCE, params, id, key);
+        }
+
+        @Override
+        public boolean validate(Integer result)
+        {
+            return true;
+        }
+
+        @Override
+        public int keyCount()
+        {
+            return result;
+        }
+    }
+
+    // Requires a custom validate() method, but fetches and stores the keys from the result set for further processing
+    protected abstract class CqlRunOpFetchKeys extends CqlRunOp<byte[][]>
+    {
+
+        protected CqlRunOpFetchKeys(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String id, ByteBuffer key)
+        {
+            super(client, query, queryId, KeysHandler.INSTANCE, params, id, key);
+        }
+
+        @Override
+        public int keyCount()
+        {
+            return result.length;
+        }
+
+    }
+
+    // Cql
+    protected abstract class CqlRunOp<V> implements RunOp
+    {
+
+        final ClientWrapper client;
+        final String query;
+        final Object queryId;
+        final List<ByteBuffer> params;
+        final String id;
+        final ByteBuffer key;
+        final ResultHandler<V> handler;
+        V result;
+
+        private CqlRunOp(ClientWrapper client, String query, Object queryId, ResultHandler<V> handler, List<ByteBuffer> params, String id, ByteBuffer key)
+        {
+            this.client = client;
+            this.query = query;
+            this.queryId = queryId;
+            this.handler = handler;
+            this.params = params;
+            this.id = id;
+            this.key = key;
+        }
+
+        @Override
+        public boolean run() throws Exception
+        {
+            return queryId != null
+            ? validate(result = client.execute(queryId, key, params, handler))
+            : validate(result = client.execute(query, key, params, handler));
+        }
+
+        @Override
+        public String key()
+        {
+            return id;
+        }
+
+        public abstract boolean validate(V result);
+
+    }
+
+
+    /// LOTS OF WRAPPING/UNWRAPPING NONSENSE
+
+
+    @Override
+    public void run(final ThriftClient client) throws IOException
+    {
+        run(wrap(client));
+    }
+
+    @Override
+    public void run(SimpleClient client) throws IOException
+    {
+        run(wrap(client));
+    }
+
+    @Override
+    public void run(JavaDriverClient client) throws IOException
+    {
+        run(wrap(client));
+    }
+
+    public ClientWrapper wrap(ThriftClient client)
+    {
+        return state.isCql3()
+                ? new Cql3CassandraClientWrapper(client)
+                : new Cql2CassandraClientWrapper(client);
+
+    }
+
+    public ClientWrapper wrap(JavaDriverClient client)
+    {
+        return new JavaDriverWrapper(client);
+    }
+
+    public ClientWrapper wrap(SimpleClient client)
+    {
+        return new SimpleClientWrapper(client);
+    }
+
+    protected interface ClientWrapper
+    {
+        Object createPreparedStatement(String cqlQuery) throws TException;
+        <V> V execute(Object preparedStatementId, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException;
+        <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException;
+    }
+
+    private final class JavaDriverWrapper implements ClientWrapper
+    {
+        final JavaDriverClient client;
+        private JavaDriverWrapper(JavaDriverClient client)
+        {
+            this.client = client;
+        }
+
+        @Override
+        public <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler)
+        {
+            String formattedQuery = formatCqlQuery(query, queryParams, state.isCql3());
+            return handler.javaDriverHandler().apply(client.execute(formattedQuery, ThriftConversion.fromThrift(state.settings.command.consistencyLevel)));
+        }
+
+        @Override
+        public <V> V execute(Object preparedStatementId, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler)
+        {
+            return handler.javaDriverHandler().apply(
+                    client.executePrepared(
+                            (PreparedStatement) preparedStatementId,
+                            queryParams,
+                            ThriftConversion.fromThrift(state.settings.command.consistencyLevel)));
+        }
+
+        @Override
+        public Object createPreparedStatement(String cqlQuery)
+        {
+            return client.prepare(cqlQuery);
+        }
+    }
+
+    private final class SimpleClientWrapper implements ClientWrapper
+    {
+        final SimpleClient client;
+        private SimpleClientWrapper(SimpleClient client)
+        {
+            this.client = client;
+        }
+
+        @Override
+        public <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler)
+        {
+            String formattedQuery = formatCqlQuery(query, queryParams, state.isCql3());
+            return handler.thriftHandler().apply(client.execute(formattedQuery, ThriftConversion.fromThrift(state.settings.command.consistencyLevel)));
+        }
+
+        @Override
+        public <V> V execute(Object preparedStatementId, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler)
+        {
+            return handler.thriftHandler().apply(
+                    client.executePrepared(
+                            (byte[]) preparedStatementId,
+                            queryParams,
+                            ThriftConversion.fromThrift(state.settings.command.consistencyLevel)));
+        }
+
+        @Override
+        public Object createPreparedStatement(String cqlQuery)
+        {
+            return client.prepare(cqlQuery).statementId.bytes;
+        }
+    }
+
+    // client wrapper for Cql3
+    private final class Cql3CassandraClientWrapper implements ClientWrapper
+    {
+        final ThriftClient client;
+        private Cql3CassandraClientWrapper(ThriftClient client)
+        {
+            this.client = client;
+        }
+
+        @Override
+        public <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException
+        {
+            String formattedQuery = formatCqlQuery(query, queryParams, true);
+            return handler.simpleNativeHandler().apply(
+                    client.execute_cql3_query(query, key, Compression.NONE, state.settings.command.consistencyLevel)
+            );
+        }
+
+        @Override
+        public <V> V execute(Object preparedStatementId, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException
+        {
+            Integer id = (Integer) preparedStatementId;
+            return handler.simpleNativeHandler().apply(
+                    client.execute_prepared_cql3_query(id, key, queryParams, state.settings.command.consistencyLevel)
+            );
+        }
+
+        @Override
+        public Object createPreparedStatement(String cqlQuery) throws TException
+        {
+            return client.prepare_cql3_query(cqlQuery, Compression.NONE);
+        }
+    }
+
+    // client wrapper for Cql2
+    private final class Cql2CassandraClientWrapper implements ClientWrapper
+    {
+        final ThriftClient client;
+        private Cql2CassandraClientWrapper(ThriftClient client)
+        {
+            this.client = client;
+        }
+
+        @Override
+        public <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException
+        {
+            String formattedQuery = formatCqlQuery(query, queryParams, false);
+            return handler.simpleNativeHandler().apply(
+                    client.execute_cql_query(formattedQuery, key, Compression.NONE)
+            );
+        }
+
+        @Override
+        public <V> V execute(Object preparedStatementId, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException
+        {
+            Integer id = (Integer) preparedStatementId;
+            return handler.simpleNativeHandler().apply(
+                    client.execute_prepared_cql_query(id, key, queryParams)
+            );
+        }
+
+        @Override
+        public Object createPreparedStatement(String cqlQuery) throws TException
+        {
+            return client.prepare_cql_query(cqlQuery, Compression.NONE);
+        }
+    }
+
+    // interface for building functions to standardise results from each client
+    protected static interface ResultHandler<V>
+    {
+        Function<ResultSet, V> javaDriverHandler();
+        Function<ResultMessage, V> thriftHandler();
+        Function<CqlResult, V> simpleNativeHandler();
+    }
+
+    protected static class RowCountHandler implements ResultHandler<Integer>
+    {
+        static final RowCountHandler INSTANCE = new RowCountHandler();
+
+        @Override
+        public Function<ResultSet, Integer> javaDriverHandler()
+        {
+            return new Function<ResultSet, Integer>()
+            {
+                @Override
+                public Integer apply(ResultSet rows)
+                {
+                    if (rows == null)
+                        return 0;
+                    return rows.all().size();
+                }
+            };
+        }
+
+        @Override
+        public Function<ResultMessage, Integer> thriftHandler()
+        {
+            return new Function<ResultMessage, Integer>()
+            {
+                @Override
+                public Integer apply(ResultMessage result)
+                {
+                    return result instanceof ResultMessage.Rows ? ((ResultMessage.Rows) result).result.size() : 0;
+                }
+            };
+        }
+
+        @Override
+        public Function<CqlResult, Integer> simpleNativeHandler()
+        {
+            return new Function<CqlResult, Integer>()
+            {
+
+                @Override
+                public Integer apply(CqlResult result)
+                {
+                    switch (result.getType())
+                    {
+                        case ROWS:
+                            return result.getRows().size();
+                        default:
+                            return 1;
+                    }
+                }
+            };
+        }
+
+    }
+
+    // Processes results from each client into an array of all key bytes returned
+    protected static final class KeysHandler implements ResultHandler<byte[][]>
+    {
+        static final KeysHandler INSTANCE = new KeysHandler();
+
+        @Override
+        public Function<ResultSet, byte[][]> javaDriverHandler()
+        {
+            return new Function<ResultSet, byte[][]>()
+            {
+
+                @Override
+                public byte[][] apply(ResultSet result)
+                {
+
+                    if (result == null)
+                        return new byte[0][];
+                    List<Row> rows = result.all();
+                    byte[][] r = new byte[rows.size()][];
+                    for (int i = 0 ; i < r.length ; i++)
+                        r[i] = rows.get(i).getBytes(0).array();
+                    return r;
+                }
+            };
+        }
+
+        @Override
+        public Function<ResultMessage, byte[][]> thriftHandler()
+        {
+            return new Function<ResultMessage, byte[][]>()
+            {
+
+                @Override
+                public byte[][] apply(ResultMessage result)
+                {
+                    if (result instanceof ResultMessage.Rows)
+                    {
+                        ResultMessage.Rows rows = ((ResultMessage.Rows) result);
+                        byte[][] r = new byte[rows.result.size()][];
+                        for (int i = 0 ; i < r.length ; i++)
+                            r[i] = rows.result.rows.get(i).get(0).array();
+                        return r;
+                    }
+                    return null;
+                }
+            };
+        }
+
+        @Override
+        public Function<CqlResult, byte[][]> simpleNativeHandler()
+        {
+            return new Function<CqlResult, byte[][]>()
+            {
+
+                @Override
+                public byte[][] apply(CqlResult result)
+                {
+                    byte[][] r = new byte[result.getRows().size()][];
+                    for (int i = 0 ; i < r.length ; i++)
+                        r[i] = result.getRows().get(i).getKey();
+                    return r;
+                }
+            };
+        }
+
+    }
+
+    private static String getUnQuotedCqlBlob(ByteBuffer term, boolean isCQL3)
+    {
+        return isCQL3
+                ? "0x" + ByteBufferUtil.bytesToHex(term)
+                : ByteBufferUtil.bytesToHex(term);
+    }
+
+    /**
+     * Constructs a CQL query string by replacing instances of the character
+     * '?', with the corresponding parameter.
+     *
+     * @param query base query string to format
+     * @param parms sequence of string query parameters
+     * @return formatted CQL query string
+     */
+    private static String formatCqlQuery(String query, List<ByteBuffer> parms, boolean isCql3)
+    {
+        int marker, position = 0;
+        StringBuilder result = new StringBuilder();
+
+        if (-1 == (marker = query.indexOf('?')) || parms.size() == 0)
+            return query;
+
+        for (ByteBuffer parm : parms)
+        {
+            result.append(query.substring(position, marker));
+            result.append(getUnQuotedCqlBlob(parm, isCql3));
+
+            position = marker + 1;
+            if (-1 == (marker = query.indexOf('?', position + 1)))
+                break;
+        }
+
+        if (position < query.length())
+            result.append(query.substring(position));
+
+        return result.toString();
+    }
+
+    protected String wrapInQuotesIfRequired(String string)
+    {
+        return state.settings.mode.cqlVersion == CqlVersion.CQL3
+                ? "\"" + string + "\""
+                : string;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java
index c01767b..467e754 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java
@@ -21,98 +21,39 @@ package org.apache.cassandra.stress.operations;
  */
 
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.thrift.Compression;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.transport.SimpleClient;
-
-public class CqlRangeSlicer extends CQLOperation
+public class CqlRangeSlicer extends CqlOperation<Integer>
 {
-    private static String cqlQuery = null;
-    private int lastRowCount;
-
-    public CqlRangeSlicer(Session client, int idx)
+    public CqlRangeSlicer(State state, long idx)
     {
-        super(client, idx);
+        super(state, idx);
     }
 
-    protected void run(CQLQueryExecutor executor) throws IOException
+    @Override
+    protected List<ByteBuffer> getQueryParameters(byte[] key)
     {
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-            throw new RuntimeException("Super columns are not implemented for CQL");
-
-        if (cqlQuery == null)
-        {
-            StringBuilder query = new StringBuilder("SELECT FIRST ").append(session.getColumnsPerKey())
-                    .append(" ''..'' FROM Standard1");
-
-            if (session.cqlVersion.startsWith("2"))
-                query.append(" USING CONSISTENCY ").append(session.getConsistencyLevel().toString());
-
-            cqlQuery = query.append(" WHERE KEY > ?").toString();
-        }
-
-        String key = String.format("%0" +  session.getTotalKeysLength() + "d", index);
-        List<String> queryParams = Collections.singletonList(getUnQuotedCqlBlob(key, session.cqlVersion.startsWith("3")));
-
-        TimerContext context = session.latency.time();
-
-        boolean success = false;
-        String exceptionMessage = null;
-
-        for (int t = 0; t < session.getRetryTimes(); t++)
-        {
-            if (success)
-                break;
+        return Collections.singletonList(ByteBuffer.wrap(key));
+    }
 
-            try
-            {
-                success = executor.execute(cqlQuery, queryParams);
-            }
-            catch (Exception e)
-            {
-                System.err.println(e);
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
-            }
-        }
+    @Override
+    protected String buildQuery()
+    {
+        StringBuilder query = new StringBuilder("SELECT FIRST ").append(state.settings.columns.maxColumnsPerKey)
+                .append(" ''..'' FROM ").append(state.settings.schema.columnFamily);
 
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error executing range slice with offset %s %s%n",
-                                index,
-                                session.getRetryTimes(),
-                                key,
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-        }
+        if (state.isCql2())
+            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
 
-        session.operations.getAndIncrement();
-        session.keys.getAndAdd(lastRowCount);
-        context.stop();
+        return query.append(" WHERE KEY > ?").toString();
     }
 
-    protected boolean validateThriftResult(CqlResult result)
+    @Override
+    protected CqlRunOp buildRunOp(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String keyid, ByteBuffer key)
     {
-        lastRowCount = result.rows.size();
-        return  lastRowCount != 0;
+        return new CqlRunOpTestNonEmpty(client, query, queryId, params, keyid, key);
     }
 
-    protected boolean validateNativeResult(ResultMessage result)
-    {
-        assert result instanceof ResultMessage.Rows;
-        lastRowCount = ((ResultMessage.Rows)result).result.size();
-        return lastRowCount != 0;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java
index 70273c1..051fd18 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java
@@ -21,116 +21,67 @@ package org.apache.cassandra.stress.operations;
  */
 
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.transport.SimpleClient;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.thrift.Compression;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.thrift.ThriftConversion;
-
-public class CqlReader extends CQLOperation
+public class CqlReader extends CqlOperation<Integer>
 {
-    private static String cqlQuery = null;
 
-    public CqlReader(Session client, int idx)
+    public CqlReader(State state, long idx)
     {
-        super(client, idx);
+        super(state, idx);
     }
 
-    protected void run(CQLQueryExecutor executor) throws IOException
+    @Override
+    protected String buildQuery()
     {
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-            throw new RuntimeException("Super columns are not implemented for CQL");
+        StringBuilder query = new StringBuilder("SELECT ");
 
-        if (cqlQuery == null)
+        if (state.settings.columns.names == null)
         {
-            StringBuilder query = new StringBuilder("SELECT ");
-
-            if (session.columnNames == null)
-            {
-                if (session.cqlVersion.startsWith("2"))
-                    query.append("FIRST ").append(session.getColumnsPerKey()).append(" ''..''");
-                else
-                    query.append("*");
-            }
+            if (state.isCql2())
+                query.append("FIRST ").append(state.settings.columns.maxColumnsPerKey).append(" ''..''");
             else
-            {
-                for (int i = 0; i < session.columnNames.size(); i++)
-                {
-                    if (i > 0) query.append(",");
-                    query.append('?');
-                }
-            }
-
-            query.append(" FROM ").append(wrapInQuotesIfRequired("Standard1"));
-
-            if (session.cqlVersion.startsWith("2"))
-                query.append(" USING CONSISTENCY ").append(session.getConsistencyLevel().toString());
-            query.append(" WHERE KEY=?");
-
-            cqlQuery = query.toString();
+                query.append("*");
         }
-
-        List<String> queryParams = new ArrayList<String>();
-        if (session.columnNames != null)
-            for (int i = 0; i < session.columnNames.size(); i++)
-                queryParams.add(getUnQuotedCqlBlob(session.columnNames.get(i).array(), session.cqlVersion.startsWith("3")));
-
-        byte[] key = generateKey();
-        queryParams.add(getUnQuotedCqlBlob(key, session.cqlVersion.startsWith("3")));
-
-        TimerContext context = session.latency.time();
-
-        boolean success = false;
-        String exceptionMessage = null;
-
-        for (int t = 0; t < session.getRetryTimes(); t++)
+        else
         {
-            if (success)
-                break;
-
-            try
-            {
-                success = executor.execute(cqlQuery, queryParams);
-            }
-            catch (Exception e)
+            for (int i = 0; i < state.settings.columns.names.size() ; i++)
             {
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
+                if (i > 0)
+                    query.append(",");
+                query.append('?');
             }
         }
 
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error reading key %s %s%n with query %s",
-                                index,
-                                session.getRetryTimes(),
-                                new String(key),
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")",
-                                cqlQuery));
-        }
+        query.append(" FROM ").append(wrapInQuotesIfRequired(state.settings.schema.columnFamily));
 
-        session.operations.getAndIncrement();
-        session.keys.getAndIncrement();
-        context.stop();
+        if (state.isCql2())
+            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
+        query.append(" WHERE KEY=?");
+        return query.toString();
     }
 
-    protected boolean validateThriftResult(CqlResult result)
+    @Override
+    protected List<ByteBuffer> getQueryParameters(byte[] key)
     {
-        return result.rows.get(0).columns.size() != 0;
+        if (state.settings.columns.names != null)
+        {
+            final List<ByteBuffer> queryParams = new ArrayList<>();
+            for (ByteBuffer name : state.settings.columns.names)
+                queryParams.add(name);
+            queryParams.add(ByteBuffer.wrap(key));
+            return queryParams;
+        }
+        return Collections.singletonList(ByteBuffer.wrap(key));
     }
 
-    protected boolean validateNativeResult(ResultMessage result)
+    @Override
+    protected CqlRunOp buildRunOp(ClientWrapper client, String query, Object queryId, List<ByteBuffer> params, String keyid, ByteBuffer key)
     {
-        return result instanceof ResultMessage.Rows && ((ResultMessage.Rows)result).result.size() != 0;
+        return new CqlRunOpTestNonEmpty(client, query, queryId, params, keyid, key);
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/IndexedRangeSlicer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/IndexedRangeSlicer.java b/tools/stress/src/org/apache/cassandra/stress/operations/IndexedRangeSlicer.java
deleted file mode 100644
index b7c72a2..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/operations/IndexedRangeSlicer.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * 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.operations;
-
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-
-public class IndexedRangeSlicer extends Operation
-{
-    private static List<ByteBuffer> values = null;
-
-    public IndexedRangeSlicer(Session client, int index)
-    {
-        super(client, index);
-    }
-
-    public void run(CassandraClient client) throws IOException
-    {
-        if (values == null)
-            values = generateValues();
-
-        String format = "%0" + session.getTotalKeysLength() + "d";
-        SlicePredicate predicate = new SlicePredicate().setSlice_range(new SliceRange(ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                                      ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                                      false, session.getColumnsPerKey()));
-
-        ColumnParent parent = new ColumnParent("Standard1");
-        int expectedPerValue = session.getNumKeys() / values.size();
-
-        ByteBuffer columnName = ByteBufferUtil.bytes("C1");
-
-        int received = 0;
-
-        String startOffset = String.format(format, 0);
-        ByteBuffer value = values.get(1); // only C1 column is indexed
-
-        IndexExpression expression = new IndexExpression(columnName, IndexOperator.EQ, value);
-
-        while (received < expectedPerValue)
-        {
-            IndexClause clause = new IndexClause(Arrays.asList(expression),
-                                                 ByteBufferUtil.bytes(startOffset),
-                                                 session.getKeysPerCall());
-
-            List<KeySlice> results = null;
-            TimerContext context = session.latency.time();
-
-            boolean success = false;
-            String exceptionMessage = null;
-
-            for (int t = 0; t < session.getRetryTimes(); t++)
-            {
-                if (success)
-                    break;
-
-                try
-                {
-                    results = client.get_indexed_slices(parent, clause, predicate, session.getConsistencyLevel());
-                    success = (results.size() != 0);
-                }
-                catch (Exception e)
-                {
-                    exceptionMessage = getExceptionMessage(e);
-                    success = false;
-                }
-            }
-
-            if (!success)
-            {
-                error(String.format("Operation [%d] retried %d times - error on calling get_indexed_slices for offset %s %s%n",
-                                    index,
-                                    session.getRetryTimes(),
-                                    startOffset,
-                                    (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-            }
-
-            received += results.size();
-
-            // convert max key found back to an integer, and increment it
-            startOffset = String.format(format, (1 + getMaxKey(results)));
-
-            session.operations.getAndIncrement();
-            session.keys.getAndAdd(results.size());
-            context.stop();
-        }
-    }
-
-    /**
-     * Get maximum key from keySlice list
-     * @param keySlices list of the KeySlice objects
-     * @return maximum key value of the list
-     */
-    private int getMaxKey(List<KeySlice> keySlices)
-    {
-        byte[] firstKey = keySlices.get(0).getKey();
-        int maxKey = ByteBufferUtil.toInt(ByteBuffer.wrap(firstKey));
-
-        for (KeySlice k : keySlices)
-        {
-            int currentKey = ByteBufferUtil.toInt(ByteBuffer.wrap(k.getKey()));
-
-            if (currentKey > maxKey)
-            {
-                maxKey = currentKey;
-            }
-        }
-
-        return maxKey;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/Inserter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/Inserter.java b/tools/stress/src/org/apache/cassandra/stress/operations/Inserter.java
deleted file mode 100644
index cbf6b98..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/operations/Inserter.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * 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.operations;
-
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-public class Inserter extends Operation
-{
-    private static List<ByteBuffer> values;
-
-    public Inserter(Session client, int index)
-    {
-        super(client, index);
-    }
-
-    public void run(CassandraClient client) throws IOException
-    {
-        if (values == null)
-            values = generateValues();
-
-        List<Column> columns = new ArrayList<Column>(session.getColumnsPerKey());
-        List<SuperColumn> superColumns = null;
-
-        // format used for keys
-        String format = "%0" + session.getTotalKeysLength() + "d";
-
-        for (int i = 0; i < session.getColumnsPerKey(); i++)
-        {
-            columns.add(new Column(columnName(i, session.timeUUIDComparator))
-                            .setValue(values.get(i % values.size()))
-                            .setTimestamp(FBUtilities.timestampMicros()));
-        }
-
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-        {
-            superColumns = new ArrayList<SuperColumn>();
-            // supers = [SuperColumn('S' + str(j), columns) for j in xrange(supers_per_key)]
-            for (int i = 0; i < session.getSuperColumns(); i++)
-            {
-                String superColumnName = "S" + Integer.toString(i);
-                superColumns.add(new SuperColumn(ByteBufferUtil.bytes(superColumnName), columns));
-            }
-        }
-
-        String rawKey = String.format(format, index);
-        Map<String, List<Mutation>> row = session.getColumnFamilyType() == ColumnFamilyType.Super
-                                        ? getSuperColumnsMutationMap(superColumns)
-                                        : getColumnsMutationMap(columns);
-        Map<ByteBuffer, Map<String, List<Mutation>>> record = Collections.singletonMap(ByteBufferUtil.bytes(rawKey), row);
-
-        TimerContext context = session.latency.time();
-
-        boolean success = false;
-        String exceptionMessage = null;
-        for (int t = 0; t < session.getRetryTimes(); t++)
-        {
-            if (success)
-                break;
-
-            try
-            {
-                client.batch_mutate(record, session.getConsistencyLevel());
-                success = true;
-            }
-            catch (Exception e)
-            {
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
-            }
-        }
-
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error inserting key %s %s%n",
-                                index,
-                                session.getRetryTimes(),
-                                rawKey,
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-        }
-
-        session.operations.getAndIncrement();
-        session.keys.getAndIncrement();
-        context.stop();
-    }
-
-    private Map<String, List<Mutation>> getSuperColumnsMutationMap(List<SuperColumn> superColumns)
-    {
-        List<Mutation> mutations = new ArrayList<Mutation>(superColumns.size());
-        for (SuperColumn s : superColumns)
-        {
-            ColumnOrSuperColumn superColumn = new ColumnOrSuperColumn().setSuper_column(s);
-            mutations.add(new Mutation().setColumn_or_supercolumn(superColumn));
-        }
-
-        return Collections.singletonMap("Super1", mutations);
-    }
-
-    private Map<String, List<Mutation>> getColumnsMutationMap(List<Column> columns)
-    {
-        List<Mutation> mutations = new ArrayList<Mutation>(columns.size());
-        for (Column c : columns)
-        {
-            ColumnOrSuperColumn column = new ColumnOrSuperColumn().setColumn(c);
-            mutations.add(new Mutation().setColumn_or_supercolumn(column));
-        }
-
-        return Collections.singletonMap("Standard1", mutations);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/MultiGetter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/MultiGetter.java b/tools/stress/src/org/apache/cassandra/stress/operations/MultiGetter.java
deleted file mode 100644
index 12a39fb..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/operations/MultiGetter.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/**
- * 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.operations;
-
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-
-public class MultiGetter extends Operation
-{
-    public MultiGetter(Session client, int index)
-    {
-        super(client, index);
-    }
-
-    public void run(CassandraClient client) throws IOException
-    {
-        SlicePredicate predicate = new SlicePredicate().setSlice_range(new SliceRange(ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                                      ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                                      false, session.getColumnsPerKey()));
-
-        int offset = index * session.getKeysPerThread();
-        Map<ByteBuffer,List<ColumnOrSuperColumn>> results;
-
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-        {
-            List<ByteBuffer> keys = generateKeys(offset, offset + session.getKeysPerCall());
-
-            for (int j = 0; j < session.getSuperColumns(); j++)
-            {
-                ColumnParent parent = new ColumnParent("Super1").setSuper_column(ByteBufferUtil.bytes("S" + j));
-
-                TimerContext context = session.latency.time();
-
-                boolean success = false;
-                String exceptionMessage = null;
-
-                for (int t = 0; t < session.getRetryTimes(); t++)
-                {
-                    if (success)
-                        break;
-
-                    try
-                    {
-                        results = client.multiget_slice(keys, parent, predicate, session.getConsistencyLevel());
-                        success = (results.size() != 0);
-                    }
-                    catch (Exception e)
-                    {
-                        exceptionMessage = getExceptionMessage(e);
-                    }
-                }
-
-                if (!success)
-                {
-                    error(String.format("Operation [%d] retried %d times - error on calling multiget_slice for keys %s %s%n",
-                                        index,
-                                        session.getRetryTimes(),
-                                        keys,
-                                        (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-                }
-
-                session.operations.getAndIncrement();
-                session.keys.getAndAdd(keys.size());
-                context.stop();
-
-                offset += session.getKeysPerCall();
-            }
-        }
-        else
-        {
-            ColumnParent parent = new ColumnParent("Standard1");
-
-            List<ByteBuffer> keys = generateKeys(offset, offset + session.getKeysPerCall());
-
-            TimerContext context = session.latency.time();
-
-            boolean success = false;
-            String exceptionMessage = null;
-
-            for (int t = 0; t < session.getRetryTimes(); t++)
-            {
-                if (success)
-                    break;
-
-                try
-                {
-                    results = client.multiget_slice(keys, parent, predicate, session.getConsistencyLevel());
-                    success = (results.size() != 0);
-                }
-                catch (Exception e)
-                {
-                    exceptionMessage = getExceptionMessage(e);
-                    success = false;
-                }
-            }
-
-            if (!success)
-            {
-                error(String.format("Operation [%d] retried %d times - error on calling multiget_slice for keys %s %s%n",
-                                    index,
-                                    session.getRetryTimes(),
-                                    keys,
-                                    (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-            }
-
-            session.operations.getAndIncrement();
-            session.keys.getAndAdd(keys.size());
-            context.stop();
-
-            offset += session.getKeysPerCall();
-        }
-    }
-
-    private List<ByteBuffer> generateKeys(int start, int limit)
-    {
-        List<ByteBuffer> keys = new ArrayList<ByteBuffer>();
-
-        for (int i = start; i < limit; i++)
-        {
-            keys.add(ByteBuffer.wrap(generateKey()));
-        }
-
-        return keys;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/RangeSlicer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/RangeSlicer.java b/tools/stress/src/org/apache/cassandra/stress/operations/RangeSlicer.java
deleted file mode 100644
index f9ba115..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/operations/RangeSlicer.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * 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.operations;
-
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-public class RangeSlicer extends Operation
-{
-
-    public RangeSlicer(Session client, int index)
-    {
-        super(client, index);
-    }
-
-    public void run(CassandraClient client) throws IOException
-    {
-        String format = "%0" + session.getTotalKeysLength() + "d";
-
-        // initial values
-        int count = session.getColumnsPerKey();
-
-        SlicePredicate predicate = new SlicePredicate().setSlice_range(new SliceRange(ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                                      ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                                      false,
-                                                                                      count));
-
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-        {
-            ByteBuffer start = ByteBufferUtil.bytes(String.format(format, index));
-
-            List<KeySlice> slices = new ArrayList<KeySlice>();
-            KeyRange range = new KeyRange(count).setStart_key(start).setEnd_key(ByteBufferUtil.EMPTY_BYTE_BUFFER);
-
-            for (int i = 0; i < session.getSuperColumns(); i++)
-            {
-                String superColumnName = "S" + Integer.toString(i);
-                ColumnParent parent = new ColumnParent("Super1").setSuper_column(ByteBufferUtil.bytes(superColumnName));
-
-                TimerContext context = session.latency.time();
-
-                boolean success = false;
-                String exceptionMessage = null;
-
-                for (int t = 0; t < session.getRetryTimes(); t++)
-                {
-                    try
-                    {
-                        slices = client.get_range_slices(parent, predicate, range, session.getConsistencyLevel());
-                        success = (slices.size() != 0);
-                    }
-                    catch (Exception e)
-                    {
-                        exceptionMessage = getExceptionMessage(e);
-                        success = false;
-                    }
-                }
-
-                if (!success)
-                {
-                    error(String.format("Operation [%d] retried %d times - error on calling get_range_slices for range offset %s %s%n",
-                                        index,
-                                        session.getRetryTimes(),
-                                        ByteBufferUtil.string(start),
-                                        (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-                }
-
-                session.operations.getAndIncrement();
-                context.stop();
-            }
-
-            session.keys.getAndAdd(slices.size());
-        }
-        else
-        {
-            ColumnParent parent = new ColumnParent("Standard1");
-
-            ByteBuffer start = ByteBufferUtil.bytes(String.format(format, index));
-
-            List<KeySlice> slices = new ArrayList<KeySlice>();
-            KeyRange range = new KeyRange(count).setStart_key(start).setEnd_key(ByteBufferUtil.EMPTY_BYTE_BUFFER);
-
-            TimerContext context = session.latency.time();
-
-            boolean success = false;
-            String exceptionMessage = null;
-
-            for (int t = 0; t < session.getRetryTimes(); t++)
-            {
-                if (success)
-                    break;
-
-                try
-                {
-                    slices = client.get_range_slices(parent, predicate, range, session.getConsistencyLevel());
-                    success = (slices.size() != 0);
-                }
-                catch (Exception e)
-                {
-                    exceptionMessage = getExceptionMessage(e);
-                    success = false;
-                }
-            }
-
-            if (!success)
-            {
-                error(String.format("Operation [%d] retried %d times - error on calling get_indexed_slices for range offset %s %s%n",
-                                    index,
-                                    session.getRetryTimes(),
-                                    ByteBufferUtil.string(start),
-                                    (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-            }
-
-            session.operations.getAndIncrement();
-            session.keys.getAndAdd(slices.size());
-            context.stop();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/Reader.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/Reader.java b/tools/stress/src/org/apache/cassandra/stress/operations/Reader.java
deleted file mode 100644
index 72d09b4..0000000
--- a/tools/stress/src/org/apache/cassandra/stress/operations/Reader.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/**
- * 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.operations;
-
-import com.yammer.metrics.core.TimerContext;
-import org.apache.cassandra.stress.Session;
-import org.apache.cassandra.stress.util.CassandraClient;
-import org.apache.cassandra.stress.util.Operation;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.thrift.*;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-public class Reader extends Operation
-{
-    public Reader(Session client, int index)
-    {
-        super(client, index);
-    }
-
-    public void run(CassandraClient client) throws IOException
-    {
-        // initialize SlicePredicate with existing SliceRange
-        SlicePredicate predicate = new SlicePredicate();
-
-        if (session.columnNames == null)
-            predicate.setSlice_range(getSliceRange());
-        else // see CASSANDRA-3064 about why this is useful
-            predicate.setColumn_names(session.columnNames);
-
-        if (session.getColumnFamilyType() == ColumnFamilyType.Super)
-        {
-            runSuperColumnReader(predicate, client);
-        }
-        else
-        {
-            runColumnReader(predicate, client);
-        }
-    }
-
-    private void runSuperColumnReader(SlicePredicate predicate, Cassandra.Client client) throws IOException
-    {
-        byte[] rawKey = generateKey();
-        ByteBuffer key = ByteBuffer.wrap(rawKey);
-
-        for (int j = 0; j < session.getSuperColumns(); j++)
-        {
-            String superColumn = 'S' + Integer.toString(j);
-            ColumnParent parent = new ColumnParent("Super1").setSuper_column(superColumn.getBytes(UTF_8));
-
-            TimerContext context = session.latency.time();
-
-            boolean success = false;
-            String exceptionMessage = null;
-
-            for (int t = 0; t < session.getRetryTimes(); t++)
-            {
-                if (success)
-                    break;
-
-                try
-                {
-                    List<ColumnOrSuperColumn> columns;
-                    columns = client.get_slice(key, parent, predicate, session.getConsistencyLevel());
-                    success = (columns.size() != 0);
-                }
-                catch (Exception e)
-                {
-                    exceptionMessage = getExceptionMessage(e);
-                    success = false;
-                }
-            }
-
-            if (!success)
-            {
-                error(String.format("Operation [%d] retried %d times - error reading key %s %s%n",
-                                    index,
-                                    session.getRetryTimes(),
-                                    new String(rawKey),
-                                    (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-            }
-
-            session.operations.getAndIncrement();
-            session.keys.getAndIncrement();
-            context.stop();
-        }
-    }
-
-    private void runColumnReader(SlicePredicate predicate, Cassandra.Client client) throws IOException
-    {
-        ColumnParent parent = new ColumnParent("Standard1");
-
-        byte[] key = generateKey();
-        ByteBuffer keyBuffer = ByteBuffer.wrap(key);
-
-        TimerContext context = session.latency.time();
-
-        boolean success = false;
-        String exceptionMessage = null;
-
-        for (int t = 0; t < session.getRetryTimes(); t++)
-        {
-            if (success)
-                break;
-
-            try
-            {
-                List<ColumnOrSuperColumn> columns;
-                columns = client.get_slice(keyBuffer, parent, predicate, session.getConsistencyLevel());
-                success = (columns.size() != 0);
-            }
-            catch (Exception e)
-            {
-                exceptionMessage = getExceptionMessage(e);
-                success = false;
-            }
-        }
-
-        if (!success)
-        {
-            error(String.format("Operation [%d] retried %d times - error reading key %s %s%n",
-                                index,
-                                session.getRetryTimes(),
-                                new String(key),
-                                (exceptionMessage == null) ? "" : "(" + exceptionMessage + ")"));
-        }
-
-        session.operations.getAndIncrement();
-        session.keys.getAndIncrement();
-        context.stop();
-    }
-
-    private SliceRange getSliceRange()
-    {
-        return new SliceRange()
-                    .setStart(new byte[] {})
-                    .setFinish(new byte[] {})
-                    .setReversed(false)
-                    .setCount(session.getColumnsPerKey());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/ThriftCounterAdder.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/ThriftCounterAdder.java b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftCounterAdder.java
new file mode 100644
index 0000000..b1657b2
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftCounterAdder.java
@@ -0,0 +1,95 @@
+/**
+ * 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.operations;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.thrift.*;
+
+public class ThriftCounterAdder extends Operation
+{
+    public ThriftCounterAdder(State state, long index)
+    {
+        super(state, index);
+        if (state.settings.columns.variableColumnCount)
+            throw new IllegalStateException("Variable column counts not supported for counters");
+    }
+
+    public void run(final ThriftClient client) throws IOException
+    {
+        List<CounterColumn> columns = new ArrayList<CounterColumn>();
+        for (int i = 0; i < state.settings.columns.maxColumnsPerKey; i++)
+            columns.add(new CounterColumn(getColumnNameBytes(i), 1L));
+
+        Map<String, List<Mutation>> row;
+        if (state.settings.columns.useSuperColumns)
+        {
+            List<Mutation> mutations = new ArrayList<>();
+            for (ColumnParent parent : state.columnParents)
+            {
+                CounterSuperColumn csc = new CounterSuperColumn(ByteBuffer.wrap(parent.getSuper_column()), columns);
+                ColumnOrSuperColumn cosc = new ColumnOrSuperColumn().setCounter_super_column(csc);
+                mutations.add(new Mutation().setColumn_or_supercolumn(cosc));
+            }
+            row = Collections.singletonMap("SuperCounter1", mutations);
+        }
+        else
+        {
+            List<Mutation> mutations = new ArrayList<>(columns.size());
+            for (CounterColumn c : columns)
+            {
+                ColumnOrSuperColumn cosc = new ColumnOrSuperColumn().setCounter_column(c);
+                mutations.add(new Mutation().setColumn_or_supercolumn(cosc));
+            }
+            row = Collections.singletonMap("Counter1", mutations);
+        }
+
+        final ByteBuffer key = getKey();
+        final Map<ByteBuffer, Map<String, List<Mutation>>> record = Collections.singletonMap(key, row);
+
+        timeWithRetry(new RunOp()
+        {
+            @Override
+            public boolean run() throws Exception
+            {
+                client.batch_mutate(record, state.settings.command.consistencyLevel);
+                return true;
+            }
+
+            @Override
+            public String key()
+            {
+                return new String(key.array());
+            }
+
+            @Override
+            public int keyCount()
+            {
+                return 1;
+            }
+        });
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/ThriftCounterGetter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/ThriftCounterGetter.java b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftCounterGetter.java
new file mode 100644
index 0000000..8567edd
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftCounterGetter.java
@@ -0,0 +1,75 @@
+/**
+ * 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.operations;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.SlicePredicate;
+import org.apache.cassandra.thrift.SliceRange;
+
+public class ThriftCounterGetter extends Operation
+{
+    public ThriftCounterGetter(State state, long index)
+    {
+        super(state, index);
+        if (state.settings.columns.variableColumnCount)
+            throw new IllegalStateException("Variable column counts not supported for counters");
+    }
+
+    public void run(final ThriftClient client) throws IOException
+    {
+        SliceRange sliceRange = new SliceRange();
+        // start/finish
+        sliceRange.setStart(new byte[] {}).setFinish(new byte[] {});
+        // reversed/count
+        sliceRange.setReversed(false).setCount(state.settings.columns.maxColumnsPerKey);
+        // initialize SlicePredicate with existing SliceRange
+        final SlicePredicate predicate = new SlicePredicate().setSlice_range(sliceRange);
+
+        final ByteBuffer key = getKey();
+        for (final ColumnParent parent : state.columnParents)
+        {
+
+            timeWithRetry(new RunOp()
+            {
+                @Override
+                public boolean run() throws Exception
+                {
+                    return client.get_slice(key, parent, predicate, state.settings.command.consistencyLevel).size() != 0;
+                }
+
+                @Override
+                public String key()
+                {
+                    return new String(key.array());
+                }
+
+                @Override
+                public int keyCount()
+                {
+                    return 1;
+                }
+            });
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/ThriftIndexedRangeSlicer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/ThriftIndexedRangeSlicer.java b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftIndexedRangeSlicer.java
new file mode 100644
index 0000000..c6b1b03
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftIndexedRangeSlicer.java
@@ -0,0 +1,115 @@
+/**
+* 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.operations;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.settings.SettingsCommandMulti;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class ThriftIndexedRangeSlicer extends Operation
+{
+    public ThriftIndexedRangeSlicer(State state, long index)
+    {
+        super(state, index);
+        if (!state.rowGen.isDeterministic() || !state.keyGen.isDeterministic())
+            throw new IllegalStateException("Only run with a isDeterministic row/key generator");
+        if (state.settings.columns.useSuperColumns || state.columnParents.size() != 1)
+            throw new IllegalStateException("Does not support super columns");
+        if (state.settings.columns.useTimeUUIDComparator)
+            throw new IllegalStateException("Does not support TimeUUID column names");
+    }
+
+    public void run(final ThriftClient client) throws IOException
+    {
+
+        final SlicePredicate predicate = new SlicePredicate()
+                .setSlice_range(new SliceRange(ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                        false, state.settings.columns.maxColumnsPerKey));
+        final List<ByteBuffer> columns = generateColumnValues();
+        final ColumnParent parent = state.columnParents.get(0);
+
+        final ByteBuffer columnName = getColumnNameBytes(1);
+        final ByteBuffer value = columns.get(1); // only C1 column is indexed
+
+        IndexExpression expression = new IndexExpression(columnName, IndexOperator.EQ, value);
+        byte[] minKey = new byte[0];
+        final List<KeySlice>[] results = new List[1];
+        do
+        {
+
+            final boolean first = minKey.length == 0;
+            final IndexClause clause = new IndexClause(Arrays.asList(expression),
+                                                 ByteBuffer.wrap(minKey),
+                                                ((SettingsCommandMulti) state.settings.command).keysAtOnce);
+
+            timeWithRetry(new RunOp()
+            {
+                @Override
+                public boolean run() throws Exception
+                {
+                    results[0] = client.get_indexed_slices(parent, clause, predicate, state.settings.command.consistencyLevel);
+                    return !first || results[0].size() > 0;
+                }
+
+                @Override
+                public String key()
+                {
+                    return new String(value.array());
+                }
+
+                @Override
+                public int keyCount()
+                {
+                    return results[0].size();
+                }
+            });
+
+            minKey = getNextMinKey(minKey, results[0]);
+
+        } while (results[0].size() > 0);
+    }
+
+    /**
+     * Get maximum key from keySlice list
+     * @param slices list of the KeySlice objects
+     * @return maximum key value of the list
+     */
+    private static byte[] getNextMinKey(byte[] cur, List<KeySlice> slices)
+    {
+        // find max
+        for (KeySlice slice : slices)
+            if (FBUtilities.compareUnsigned(cur, slice.getKey()) < 0)
+                cur = slice.getKey();
+
+        // increment
+        for (int i = 0 ; i < cur.length ; i++)
+            if (++cur[i] != 0)
+                break;
+        return cur;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/ThriftInserter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/ThriftInserter.java b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftInserter.java
new file mode 100644
index 0000000..c5f8051
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftInserter.java
@@ -0,0 +1,117 @@
+/**
+ * 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.operations;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.UUIDGen;
+
+public final class ThriftInserter extends Operation
+{
+
+    public ThriftInserter(State state, long index)
+    {
+        super(state, index);
+    }
+
+    public void run(final ThriftClient client) throws IOException
+    {
+        final ByteBuffer key = getKey();
+        final List<Column> columns = generateColumns();
+
+        Map<String, List<Mutation>> row;
+        if (!state.settings.columns.useSuperColumns)
+        {
+            List<Mutation> mutations = new ArrayList<>(columns.size());
+            for (Column c : columns)
+            {
+                ColumnOrSuperColumn column = new ColumnOrSuperColumn().setColumn(c);
+                mutations.add(new Mutation().setColumn_or_supercolumn(column));
+            }
+            row = Collections.singletonMap(state.settings.schema.columnFamily, mutations);
+        }
+        else
+        {
+            List<Mutation> mutations = new ArrayList<>(state.columnParents.size());
+            for (ColumnParent parent : state.columnParents)
+            {
+                final SuperColumn s = new SuperColumn(parent.bufferForSuper_column(), columns);
+                final ColumnOrSuperColumn cosc = new ColumnOrSuperColumn().setSuper_column(s);
+                mutations.add(new Mutation().setColumn_or_supercolumn(cosc));
+            }
+            row = Collections.singletonMap("Super1", mutations);
+        }
+
+        final Map<ByteBuffer, Map<String, List<Mutation>>> record = Collections.singletonMap(key, row);
+
+        timeWithRetry(new RunOp()
+        {
+            @Override
+            public boolean run() throws Exception
+            {
+                client.batch_mutate(record, state.settings.command.consistencyLevel);
+                return true;
+            }
+
+            @Override
+            public String key()
+            {
+                return new String(key.array());
+            }
+
+            @Override
+            public int keyCount()
+            {
+                return 1;
+            }
+        });
+    }
+
+    protected List<Column> generateColumns()
+    {
+        final List<ByteBuffer> values = generateColumnValues();
+        final List<Column> columns = new ArrayList<>(values.size());
+
+        if (state.settings.columns.useTimeUUIDComparator)
+            for (int i = 0 ; i < values.size() ; i++)
+                new Column(TimeUUIDType.instance.decompose(UUIDGen.getTimeUUID()));
+        else
+            // TODO : consider randomly allocating column names in case where have fewer than max columns
+            // but need to think about implications for indexes / indexed range slicer / other knock on effects
+            for (int i = 0 ; i < values.size() ; i++)
+                columns.add(new Column(getColumnNameBytes(i)));
+
+        for (int i = 0 ; i < values.size() ; i++)
+            columns.get(i)
+                    .setValue(values.get(i))
+                    .setTimestamp(FBUtilities.timestampMicros());
+
+        return columns;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/ThriftMultiGetter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/ThriftMultiGetter.java b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftMultiGetter.java
new file mode 100644
index 0000000..01c7325
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftMultiGetter.java
@@ -0,0 +1,81 @@
+/**
+ * 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.operations;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.settings.SettingsCommandMulti;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.SlicePredicate;
+import org.apache.cassandra.thrift.SliceRange;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+public final class ThriftMultiGetter extends Operation
+{
+
+    public ThriftMultiGetter(State state, long index)
+    {
+        super(state, index);
+    }
+
+    public void run(final ThriftClient client) throws IOException
+    {
+
+        final SlicePredicate predicate = new SlicePredicate().setSlice_range(
+                new SliceRange(
+                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                        false,
+                        state.settings.columns.maxColumnsPerKey
+                )
+        );
+
+        final List<ByteBuffer> keys = getKeys(((SettingsCommandMulti) state.settings.command).keysAtOnce);
+
+        for (final ColumnParent parent : state.columnParents)
+        {
+            timeWithRetry(new RunOp()
+            {
+                int count;
+                @Override
+                public boolean run() throws Exception
+                {
+                    return (count = client.multiget_slice(keys, parent, predicate, state.settings.command.consistencyLevel).size()) != 0;
+                }
+
+                @Override
+                public String key()
+                {
+                    return keys.toString();
+                }
+
+                @Override
+                public int keyCount()
+                {
+                    return count;
+                }
+            });
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/ThriftRangeSlicer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/ThriftRangeSlicer.java b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftRangeSlicer.java
new file mode 100644
index 0000000..ce6c8cd
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftRangeSlicer.java
@@ -0,0 +1,86 @@
+/**
+ * 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.operations;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.settings.SettingsCommandMulti;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.KeyRange;
+import org.apache.cassandra.thrift.SlicePredicate;
+import org.apache.cassandra.thrift.SliceRange;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public final class ThriftRangeSlicer extends Operation
+{
+
+    public ThriftRangeSlicer(State state, long index)
+    {
+        super(state, index);
+    }
+
+    @Override
+    public void run(final ThriftClient client) throws IOException
+    {
+        final SlicePredicate predicate = new SlicePredicate()
+                .setSlice_range(
+                        new SliceRange(
+                                ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                false,
+                                state.settings.columns.maxColumnsPerKey
+                        )
+                );
+
+        final ByteBuffer start = getKey();
+        final KeyRange range =
+                new KeyRange(state.settings.columns.maxColumnsPerKey)
+                        .setStart_key(start)
+                        .setEnd_key(ByteBufferUtil.EMPTY_BYTE_BUFFER)
+                        .setCount(((SettingsCommandMulti)state.settings.command).keysAtOnce);
+
+        for (final ColumnParent parent : state.columnParents)
+        {
+            timeWithRetry(new RunOp()
+            {
+                private int count = 0;
+                @Override
+                public boolean run() throws Exception
+                {
+                    return (count = client.get_range_slices(parent, predicate, range, state.settings.command.consistencyLevel).size()) != 0;
+                }
+
+                @Override
+                public String key()
+                {
+                    return new String(range.bufferForStart_key().array());
+                }
+
+                @Override
+                public int keyCount()
+                {
+                    return count;
+                }
+            });
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2e1e98ad/tools/stress/src/org/apache/cassandra/stress/operations/ThriftReader.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/ThriftReader.java b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftReader.java
new file mode 100644
index 0000000..a8605e8
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/ThriftReader.java
@@ -0,0 +1,76 @@
+/**
+ * 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.operations;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.util.ThriftClient;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.SlicePredicate;
+import org.apache.cassandra.thrift.SliceRange;
+
+public final class ThriftReader extends Operation
+{
+
+    public ThriftReader(State state, long index)
+    {
+        super(state, index);
+    }
+
+    public void run(final ThriftClient client) throws IOException
+    {
+        final SlicePredicate predicate = new SlicePredicate();
+        if (state.settings.columns.names == null)
+            predicate.setSlice_range(new SliceRange()
+                    .setStart(new byte[] {})
+                    .setFinish(new byte[] {})
+                    .setReversed(false)
+                    .setCount(state.settings.columns.maxColumnsPerKey)
+            );
+        else // see CASSANDRA-3064 about why this is useful
+            predicate.setColumn_names(state.settings.columns.names);
+
+        final ByteBuffer key = getKey();
+        for (final ColumnParent parent : state.columnParents)
+        {
+            timeWithRetry(new RunOp()
+            {
+                @Override
+                public boolean run() throws Exception
+                {
+                    return client.get_slice(key, parent, predicate, state.settings.command.consistencyLevel).size() != 0;
+                }
+
+                @Override
+                public String key()
+                {
+                    return new String(key.array());
+                }
+
+                @Override
+                public int keyCount()
+                {
+                    return 1;
+                }
+            });
+        }
+    }
+
+}