You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2018/09/01 08:44:17 UTC

[1/7] cassandra git commit: Log keyspace in full query log

Repository: cassandra
Updated Branches:
  refs/heads/trunk f7431b432 -> f83bd5ac2


Log keyspace in full query log

Patch by marcuse; reviewed by Dinesh Joshi for CASSANDRA-14656


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

Branch: refs/heads/trunk
Commit: 46c33f324e5f3373d85838f364aece7ca6a6189c
Parents: f7431b4
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Aug 20 18:30:21 2018 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Sat Sep 1 08:23:54 2018 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/audit/FullQueryLogger.java | 13 +++-
 .../cassandra/audit/FullQueryLoggerTest.java    | 72 +++++++++++++++++++-
 3 files changed, 82 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/46c33f32/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b53b986..cd2a14a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Log keyspace in full query log (CASSANDRA-14656)
  * Transient Replication and Cheap Quorums (CASSANDRA-14404)
  * Log server-generated timestamp and nowInSeconds used by queries in FQL (CASSANDRA-14675)
  * Add diagnostic events for read repairs (CASSANDRA-14668)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/46c33f32/src/java/org/apache/cassandra/audit/FullQueryLogger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/audit/FullQueryLogger.java b/src/java/org/apache/cassandra/audit/FullQueryLogger.java
index 8cd8f4a..c9f8447 100644
--- a/src/java/org/apache/cassandra/audit/FullQueryLogger.java
+++ b/src/java/org/apache/cassandra/audit/FullQueryLogger.java
@@ -21,6 +21,8 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import javax.annotation.Nullable;
+
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Ints;
 
@@ -53,6 +55,7 @@ public class FullQueryLogger extends BinLogAuditLogger implements IAuditLogger
 
     public static final String GENERATED_TIMESTAMP = "generated-timestamp";
     public static final String GENERATED_NOW_IN_SECONDS = "generated-now-in-seconds";
+    public static final String KEYSPACE = "keyspace";
 
     public static final String BATCH = "batch";
     public static final String SINGLE_QUERY = "single-query";
@@ -262,6 +265,8 @@ public class FullQueryLogger extends BinLogAuditLogger implements IAuditLogger
 
         private final long generatedTimestamp;
         private final int generatedNowInSeconds;
+        @Nullable
+        private final String keyspace;
 
         AbstractLogEntry(QueryOptions queryOptions, QueryState queryState, long queryStartTime)
         {
@@ -273,6 +278,7 @@ public class FullQueryLogger extends BinLogAuditLogger implements IAuditLogger
 
             this.generatedTimestamp = queryState.generatedTimestamp();
             this.generatedNowInSeconds = queryState.generatedNowInSeconds();
+            this.keyspace = queryState.getClientState().getRawKeyspace();
 
             /*
              * Struggled with what tradeoff to make in terms of query options which is potentially large and complicated
@@ -309,6 +315,8 @@ public class FullQueryLogger extends BinLogAuditLogger implements IAuditLogger
 
             wire.write(GENERATED_TIMESTAMP).int64(generatedTimestamp);
             wire.write(GENERATED_NOW_IN_SECONDS).int32(generatedNowInSeconds);
+
+            wire.write(KEYSPACE).text(keyspace);
         }
 
         @Override
@@ -325,7 +333,10 @@ public class FullQueryLogger extends BinLogAuditLogger implements IAuditLogger
                  + 4                                                  // protocolVersion
                  + EMPTY_BYTEBUF_SIZE + queryOptionsBuffer.capacity() // queryOptionsBuffer
                  + 8                                                  // generatedTimestamp
-                 + 4;                                                 // generatedNowInSeconds
+                 + 4                                                  // generatedNowInSeconds
+                 + (keyspace != null
+                    ? Ints.checkedCast(ObjectSizes.sizeOf(keyspace))  // keyspace
+                    : OBJECT_REFERENCE_SIZE);                         // null
         }
 
         protected abstract String type();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/46c33f32/test/unit/org/apache/cassandra/audit/FullQueryLoggerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/audit/FullQueryLoggerTest.java b/test/unit/org/apache/cassandra/audit/FullQueryLoggerTest.java
index 542f8bb..376bde5 100644
--- a/test/unit/org/apache/cassandra/audit/FullQueryLoggerTest.java
+++ b/test/unit/org/apache/cassandra/audit/FullQueryLoggerTest.java
@@ -28,6 +28,8 @@ import java.util.List;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import javax.annotation.Nullable;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -55,6 +57,7 @@ import org.apache.cassandra.utils.binlog.BinLogTest;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import static org.apache.cassandra.audit.FullQueryLogger.BATCH;
@@ -416,6 +419,20 @@ public class FullQueryLoggerTest extends CQLTester
         configureFQL();
         logQuery("foo");
         Util.spinAssertEquals(true, () -> checkForQueries(Arrays.asList("foo")), 60);
+        assertRoundTripQuery(null);
+    }
+
+    @Test
+    public void testRoundTripQueryWithKeyspace() throws Exception
+    {
+        configureFQL();
+        logQuery("foo", "abcdefg");
+        Util.spinAssertEquals(true, () -> checkForQueries(Arrays.asList("foo")), 60);
+        assertRoundTripQuery("abcdefg");
+    }
+
+    private void assertRoundTripQuery(@Nullable String keyspace)
+    {
         try (ChronicleQueue queue = ChronicleQueueBuilder.single(tempDir.toFile()).rollCycle(RollCycles.TEST_SECONDLY).build())
         {
             ExcerptTailer tailer = queue.createTailer();
@@ -432,19 +449,46 @@ public class FullQueryLoggerTest extends CQLTester
                 QueryOptions queryOptions = QueryOptions.codec.decode(Unpooled.wrappedBuffer(wire.read(QUERY_OPTIONS).bytes()), protocolVersion);
                 compareQueryOptions(QueryOptions.DEFAULT, queryOptions);
 
+                String wireKeyspace = wire.read(FullQueryLogger.KEYSPACE).text();
+                assertEquals(keyspace, wireKeyspace);
+
                 assertEquals("foo", wire.read(QUERY).text());
             }));
         }
     }
 
     @Test
-    public void testRoundTripBatch() throws Exception
+    public void testRoundTripBatchWithKeyspace() throws Exception
+    {
+        configureFQL();
+        instance.logBatch(Type.UNLOGGED,
+                          Arrays.asList("foo1", "foo2"),
+                          Arrays.asList(Arrays.asList(ByteBuffer.allocate(1),
+                                                      ByteBuffer.allocateDirect(2)),
+                                        Collections.emptyList()),
+                          QueryOptions.DEFAULT,
+                          queryState("abcdefgh"),
+                          1);
+
+        Util.spinAssertEquals(true, () ->
+        {
+            try (ChronicleQueue queue = ChronicleQueueBuilder.single(tempDir.toFile()).rollCycle(RollCycles.TEST_SECONDLY).build())
+            {
+                return queue.createTailer().readingDocument().isPresent();
+            }
+        }, 60);
+
+        assertRoundTripBatch("abcdefgh");
+    }
+
+    @Test
+    public void testRoundTripBatchWithKeyspaceNull() throws Exception
     {
         configureFQL();
         instance.logBatch(Type.UNLOGGED,
                           Arrays.asList("foo1", "foo2"),
                           Arrays.asList(Arrays.asList(ByteBuffer.allocate(1),
-                                        ByteBuffer.allocateDirect(2)),
+                                                      ByteBuffer.allocateDirect(2)),
                                         Collections.emptyList()),
                           QueryOptions.DEFAULT,
                           queryState(),
@@ -458,6 +502,12 @@ public class FullQueryLoggerTest extends CQLTester
             }
         }, 60);
 
+        assertRoundTripBatch(null);
+    }
+
+
+    private void assertRoundTripBatch(@Nullable String keyspace)
+    {
         try (ChronicleQueue queue = ChronicleQueueBuilder.single(tempDir.toFile()).rollCycle(RollCycles.TEST_SECONDLY).build())
         {
             ExcerptTailer tailer = queue.createTailer();
@@ -475,7 +525,7 @@ public class FullQueryLoggerTest extends CQLTester
 
                 assertEquals(Long.MIN_VALUE, wire.read(GENERATED_TIMESTAMP).int64());
                 assertEquals(Integer.MIN_VALUE, wire.read(GENERATED_NOW_IN_SECONDS).int32());
-
+                assertEquals(keyspace, wire.read(FullQueryLogger.KEYSPACE).text());
                 assertEquals("UNLOGGED", wire.read(BATCH_TYPE).text());
                 ValueIn in = wire.read(QUERIES);
                 assertEquals(2, in.int32());
@@ -491,6 +541,7 @@ public class FullQueryLoggerTest extends CQLTester
         }
     }
 
+
     @Test
     public void testQueryWeight()
     {
@@ -522,6 +573,10 @@ public class FullQueryLoggerTest extends CQLTester
         Batch batch = new Batch(Type.UNLOGGED, new ArrayList<>(), new ArrayList<>(), QueryOptions.DEFAULT, queryState(), 1);
         assertTrue(batch.weight() > 0);
 
+        // make sure that a batch with keyspace set has a higher weight
+        Batch batch2 = new Batch(Type.UNLOGGED, new ArrayList<>(), new ArrayList<>(), QueryOptions.DEFAULT, queryState("ABABA"), 1);
+        assertTrue(batch.weight() < batch2.weight());
+
         StringBuilder sb = new StringBuilder();
         for (int ii = 0; ii < 1024 * 1024; ii++)
         {
@@ -642,6 +697,17 @@ public class FullQueryLoggerTest extends CQLTester
         instance.logQuery(query, QueryOptions.DEFAULT, queryState(), 1);
     }
 
+    private void logQuery(String query, String keyspace)
+    {
+        instance.logQuery(query, QueryOptions.DEFAULT, queryState(keyspace), 1);
+    }
+
+    private QueryState queryState(String keyspace)
+    {
+        ClientState clientState = ClientState.forInternalCalls(keyspace);
+        return new QueryState(clientState);
+    }
+
     private QueryState queryState()
     {
         return new QueryState(ClientState.forInternalCalls());


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[7/7] cassandra git commit: Add fqltool compare

Posted by ma...@apache.org.
Add fqltool compare

Patch by marcuse; reviewed by Jason Brown and Dinesh Joshi for CASSANDRA-14619


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

Branch: refs/heads/trunk
Commit: f83bd5ac2bbc6755213a6ad0675e7e5400c79670
Parents: 62ffb77
Author: Marcus Eriksson <ma...@apache.org>
Authored: Fri Aug 24 14:41:09 2018 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Sat Sep 1 09:59:21 2018 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 bin/fqltool                                     |  76 --
 bin/fqltool.bat                                 |  36 -
 build.xml                                       |  67 +-
 ide/idea-iml-file.xml                           |   2 +
 ide/idea/workspace.xml                          |   1 +
 .../cassandra/tools/FullQueryLogTool.java       |  95 ---
 .../tools/fqltool/DriverResultSet.java          | 241 ------
 .../cassandra/tools/fqltool/FQLQuery.java       | 278 -------
 .../tools/fqltool/FQLQueryIterator.java         |  72 --
 .../cassandra/tools/fqltool/FQLQueryReader.java | 116 ---
 .../cassandra/tools/fqltool/QueryReplayer.java  | 167 ----
 .../tools/fqltool/ResultComparator.java         | 116 ---
 .../cassandra/tools/fqltool/ResultHandler.java  | 124 ---
 .../cassandra/tools/fqltool/ResultStore.java    | 142 ----
 .../cassandra/tools/fqltool/commands/Dump.java  | 325 --------
 .../tools/fqltool/commands/Replay.java          | 148 ----
 .../cassandra/tools/fqltool/FQLReplayTest.java  | 760 -------------------
 tools/bin/cassandra.in.bat                      |   2 +-
 tools/bin/cassandra.in.sh                       |   2 +-
 tools/bin/fqltool                               |  76 ++
 tools/bin/fqltool.bat                           |  36 +
 .../cassandra/fqltool/DriverResultSet.java      | 248 ++++++
 .../org/apache/cassandra/fqltool/FQLQuery.java  | 265 +++++++
 .../cassandra/fqltool/FQLQueryIterator.java     |  72 ++
 .../cassandra/fqltool/FQLQueryReader.java       | 116 +++
 .../cassandra/fqltool/FullQueryLogTool.java     |  99 +++
 .../apache/cassandra/fqltool/QueryReplayer.java | 172 +++++
 .../cassandra/fqltool/ResultComparator.java     | 116 +++
 .../apache/cassandra/fqltool/ResultHandler.java | 133 ++++
 .../apache/cassandra/fqltool/ResultStore.java   | 291 +++++++
 .../cassandra/fqltool/StoredResultSet.java      | 292 +++++++
 .../cassandra/fqltool/commands/Compare.java     | 120 +++
 .../apache/cassandra/fqltool/commands/Dump.java | 325 ++++++++
 .../cassandra/fqltool/commands/Replay.java      | 148 ++++
 .../cassandra/fqltool/FQLCompareTest.java       | 131 ++++
 .../apache/cassandra/fqltool/FQLReplayTest.java | 675 ++++++++++++++++
 37 files changed, 3384 insertions(+), 2702 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1227337..1ba9975 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Add fqltool compare (CASSANDRA-14619)
  * Add fqltool replay (CASSANDRA-14618)
  * Log keyspace in full query log (CASSANDRA-14656)
  * Transient Replication and Cheap Quorums (CASSANDRA-14404)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/bin/fqltool
----------------------------------------------------------------------
diff --git a/bin/fqltool b/bin/fqltool
deleted file mode 100755
index 15a0b20..0000000
--- a/bin/fqltool
+++ /dev/null
@@ -1,76 +0,0 @@
-#!/bin/sh
-
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-if [ "x$CASSANDRA_INCLUDE" = "x" ]; then
-    # Locations (in order) to use when searching for an include file.
-    for include in "`dirname "$0"`/cassandra.in.sh" \
-                   "$HOME/.cassandra.in.sh" \
-                   /usr/share/cassandra/cassandra.in.sh \
-                   /usr/local/share/cassandra/cassandra.in.sh \
-                   /opt/cassandra/cassandra.in.sh; do
-        if [ -r "$include" ]; then
-            . "$include"
-            break
-        fi
-    done
-elif [ -r "$CASSANDRA_INCLUDE" ]; then
-    . "$CASSANDRA_INCLUDE"
-fi
-
-if [ -z "$CASSANDRA_CONF" -o -z "$CLASSPATH" ]; then
-    echo "You must set the CASSANDRA_CONF and CLASSPATH vars" >&2
-    exit 1
-fi
-
-# Run cassandra-env.sh to pick up JMX_PORT
-if [ -f "$CASSANDRA_CONF/cassandra-env.sh" ]; then
-    JVM_OPTS_SAVE=$JVM_OPTS
-    MAX_HEAP_SIZE_SAVE=$MAX_HEAP_SIZE
-    . "$CASSANDRA_CONF/cassandra-env.sh"
-    MAX_HEAP_SIZE=$MAX_HEAP_SIZE_SAVE
-    JVM_OPTS=$JVM_OPTS_SAVE
-fi
-
-# JMX Port passed via cmd line args (-p 9999 / --port 9999 / --port=9999)
-# should override the value from cassandra-env.sh
-ARGS=""
-JVM_ARGS=""
-while true
-do
-  if [ ! $1 ]; then break; fi
-  case $1 in
-    -D*)
-      JVM_ARGS="$JVM_ARGS $1"
-      ;;
-    *)
-      ARGS="$ARGS $1"
-      ;;
-  esac
-  shift
-done
-
-if [ "x$MAX_HEAP_SIZE" = "x" ]; then
-    MAX_HEAP_SIZE="512m"
-fi
-
-"$JAVA" $JAVA_AGENT -ea -da:net.openhft... -cp "$CLASSPATH" $JVM_OPTS -Xmx$MAX_HEAP_SIZE \
-        -Dlog4j.configurationFile=log4j2-tools.xml \
-        $JVM_ARGS \
-        org.apache.cassandra.tools.FullQueryLogTool $ARGS
-
-# vi:ai sw=4 ts=4 tw=0 et

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/bin/fqltool.bat
----------------------------------------------------------------------
diff --git a/bin/fqltool.bat b/bin/fqltool.bat
deleted file mode 100644
index f3103d8..0000000
--- a/bin/fqltool.bat
+++ /dev/null
@@ -1,36 +0,0 @@
-@REM
-@REM Licensed to the Apache Software Foundation (ASF) under one or more
-@REM contributor license agreements. See the NOTICE file distributed with
-@REM this work for additional information regarding copyright ownership.
-@REM The ASF licenses this file to You under the Apache License, Version 2.0
-@REM (the "License"); you may not use this file except in compliance with
-@REM the License. You may obtain a copy of the License at
-@REM
-@REM http://www.apache.org/licenses/LICENSE-2.0
-@REM
-@REM Unless required by applicable law or agreed to in writing, software
-@REM distributed under the License is distributed on an "AS IS" BASIS,
-@REM WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-@REM See the License for the specific language governing permissions and
-@REM limitations under the License.
-
-@echo off
-if "%OS%" == "Windows_NT" setlocal
-
-pushd "%~dp0"
-call cassandra.in.bat
-
-if NOT DEFINED JAVA_HOME goto :err
-
-set CASSANDRA_PARAMS=%CASSANDRA_PARAMS% -Dcassandra.logdir="%CASSANDRA_HOME%\logs"
-
-"%JAVA_HOME%\bin\java" -cp %CASSANDRA_CLASSPATH% %CASSANDRA_PARAMS% -Dlog4j.configurationFile=log4j2-tools.xml org.apache.cassandra.tools.FullQueryLogTool %*
-goto finally
-
-:err
-echo The JAVA_HOME environment variable must be set to run this program!
-pause
-
-:finally
-ENDLOCAL & set RC=%ERRORLEVEL%
-exit /B %RC%

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index b53c47b..1c957d3 100644
--- a/build.xml
+++ b/build.xml
@@ -233,6 +233,7 @@
         <mkdir dir="${test.lib}"/>
         <mkdir dir="${test.classes}"/>
         <mkdir dir="${stress.test.classes}"/>
+        <mkdir dir="${fqltool.test.classes}"/>
         <mkdir dir="${build.src.gen-java}"/>
         <mkdir dir="${build.dir.lib}"/>
         <mkdir dir="${jacoco.export.dir}"/>
@@ -914,6 +915,50 @@
         </testmacro>
     </target>
 
+    <!--
+        fqltool build file
+        -->
+    <property name="fqltool.build.src" value="${basedir}/tools/fqltool/src" />
+    <property name="fqltool.test.src" value="${basedir}/tools/fqltool/test/unit" />
+    <property name="fqltool.build.classes" value="${build.classes}/fqltool" />
+    <property name="fqltool.test.classes" value="${build.dir}/test/fqltool-classes" />
+    <property name="fqltool.manifest" value="${fqltool.build.classes}/MANIFEST.MF" />
+
+    <target name="fqltool-build-test" depends="fqltool-build" description="Compile fqltool tests">
+        <javac debug="true" debuglevel="${debuglevel}" destdir="${fqltool.test.classes}"
+               source="${source.version}" target="${target.version}"
+               includeantruntime="false" encoding="utf-8">
+            <classpath>
+                <path refid="cassandra.classpath"/>
+                <pathelement location="${fqltool.build.classes}" />
+            </classpath>
+            <src path="${fqltool.test.src}"/>
+        </javac>
+    </target>
+
+    <target name="fqltool-build" depends="build" description="build fqltool">
+    	<mkdir dir="${fqltool.build.classes}" />
+        <javac compiler="modern" debug="true" debuglevel="${debuglevel}"
+               source="${source.version}" target="${target.version}"
+               encoding="utf-8" destdir="${fqltool.build.classes}" includeantruntime="true">
+            <src path="${fqltool.build.src}" />
+            <classpath>
+                <path refid="cassandra.classes" />
+                <path>
+                    <fileset dir="${build.lib}">
+                        <include name="**/*.jar" />
+                    </fileset>
+                </path>
+            </classpath>
+        </javac>
+    </target>
+
+    <target name="fqltool-test" depends="fqltool-build-test, build-test" description="Runs fqltool tests">
+        <testmacro inputdir="${fqltool.test.src}"
+                       timeout="${test.timeout}">
+        </testmacro>
+    </target>
+
 	<target name="_write-poms" depends="maven-declare-dependencies">
 	    <artifact:writepom pomRefId="parent-pom" file="${build.dir}/${final.name}-parent.pom"/>
 	    <artifact:writepom pomRefId="all-pom" file="${build.dir}/${final.name}.pom"/>
@@ -950,7 +995,7 @@
       </jar>
     </target>
     <target name="jar"
-            depends="_main-jar, build-test, stress-build, write-poms"
+            depends="_main-jar, build-test, stress-build, fqltool-build, write-poms"
             description="Assemble Cassandra JAR files">
       <!-- Stress jar -->
       <manifest file="${stress.manifest}">
@@ -962,6 +1007,16 @@
       <jar destfile="${build.dir}/tools/lib/stress.jar" manifest="${stress.manifest}">
         <fileset dir="${stress.build.classes}"/>
       </jar>
+      <!-- fqltool jar -->
+      <manifest file="${fqltool.manifest}">
+        <attribute name="Built-By" value="Marcus Eriksson"/>
+        <attribute name="Main-Class" value="org.apache.cassandra.fqltool.FullQueryLogTool"/>
+      </manifest>
+      <mkdir dir="${fqltool.build.classes}/META-INF" />
+      <mkdir dir="${build.dir}/tools/lib/" />
+      <jar destfile="${build.dir}/tools/lib/fqltool.jar" manifest="${stress.manifest}">
+        <fileset dir="${fqltool.build.classes}"/>
+      </jar>
     </target>
 
     <!--
@@ -1178,7 +1233,7 @@
     <!-- use https://github.com/krummas/jstackjunit to get thread dumps when unit tests time out -->
   <taskdef name="junit" classname="org.krummas.junit.JStackJUnitTask" classpath="lib/jstackjunit-0.0.1.jar"/>
 
-  <target name="build-test" depends="_main-jar, stress-build, write-poms" description="Compile test classes">
+  <target name="build-test" depends="_main-jar, stress-build, fqltool-build, write-poms" description="Compile test classes">
     <javac
      compiler="modern"
      debug="true"
@@ -1257,9 +1312,11 @@
         <classpath>
           <pathelement path="${java.class.path}"/>
           <pathelement location="${stress.build.classes}"/>
+          <pathelement location="${fqltool.build.classes}"/>
           <path refid="cassandra.classpath.test" />
           <pathelement location="${test.classes}"/>
           <pathelement location="${stress.test.classes}"/>
+          <pathelement location="${fqltool.test.classes}"/>
           <pathelement location="${test.conf}"/>
           <fileset dir="${test.lib}">
             <include name="**/*.jar" />
@@ -1392,7 +1449,7 @@
     </testmacro>
   </target>
 
-  <target name="test-compression" depends="build-test, stress-build" description="Execute unit tests with sstable compression enabled">
+  <target name="test-compression" depends="build-test, stress-build, fqltool-build" description="Execute unit tests with sstable compression enabled">
     <property name="compressed_yaml" value="${build.test.dir}/cassandra.compressed.yaml"/>
     <concat destfile="${compressed_yaml}">
       <fileset file="${test.conf}/cassandra.yaml"/>
@@ -1529,7 +1586,7 @@
   </target>
 
   <target name="test-all"
-          depends="eclipse-warnings,test,long-test,test-compression,stress-test"
+          depends="eclipse-warnings,test,long-test,test-compression,stress-test,fqltool-test"
           description="Run all tests except for those under test-burn" />
 
   <!-- Use JaCoCo ant extension without needing externally saved lib -->
@@ -1848,7 +1905,9 @@
   <classpathentry kind="src" output="build/test/classes" path="test/long"/>
   <classpathentry kind="src" output="build/test/classes" path="test/resources" />
   <classpathentry kind="src" path="tools/stress/src"/>
+  <classpathentry kind="src" path="tools/fqltool/src"/>
   <classpathentry kind="src" output="build/test/stress-classes" path="tools/stress/test/unit" />
+  <classpathentry kind="src" output="build/test/fqltool-classes" path="tools/fqltool/test/unit" />
   <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER"/>
   <classpathentry kind="output" path="build/classes/eclipse"/>
   <classpathentry kind="lib" path="test/conf"/>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/ide/idea-iml-file.xml
----------------------------------------------------------------------
diff --git a/ide/idea-iml-file.xml b/ide/idea-iml-file.xml
index 2512b1d..b83abfa 100644
--- a/ide/idea-iml-file.xml
+++ b/ide/idea-iml-file.xml
@@ -29,6 +29,8 @@
             <sourceFolder url="file://$MODULE_DIR$/src/resources" type="java-resource" />
             <sourceFolder url="file://$MODULE_DIR$/tools/stress/src" isTestSource="false" />
             <sourceFolder url="file://$MODULE_DIR$/tools/stress/test/unit" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/tools/fqltool/src" isTestSource="false" />
+            <sourceFolder url="file://$MODULE_DIR$/tools/fqltool/test/unit" isTestSource="true" />
             <sourceFolder url="file://$MODULE_DIR$/test/unit" isTestSource="true" />
             <sourceFolder url="file://$MODULE_DIR$/test/long" isTestSource="true" />
             <sourceFolder url="file://$MODULE_DIR$/test/microbench" isTestSource="true" />

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/ide/idea/workspace.xml
----------------------------------------------------------------------
diff --git a/ide/idea/workspace.xml b/ide/idea/workspace.xml
index 876ed58..a2dea2a 100644
--- a/ide/idea/workspace.xml
+++ b/ide/idea/workspace.xml
@@ -297,6 +297,7 @@
         <filter targetName="codecoverage" isVisible="true" />
         <filter targetName="build-project" isVisible="false" />
         <filter targetName="stress-build" isVisible="true" />
+        <filter targetName="fqltool-build" isVisible="true" />
         <filter targetName="_write-poms" isVisible="false" />
         <filter targetName="write-poms" isVisible="false" />
         <filter targetName="jar" isVisible="true" />

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/FullQueryLogTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/FullQueryLogTool.java b/src/java/org/apache/cassandra/tools/FullQueryLogTool.java
deleted file mode 100644
index c1d4713..0000000
--- a/src/java/org/apache/cassandra/tools/FullQueryLogTool.java
+++ /dev/null
@@ -1,95 +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.tools;
-
-import java.util.List;
-
-import com.google.common.base.Throwables;
-
-import io.airlift.airline.Cli;
-import io.airlift.airline.Help;
-import io.airlift.airline.ParseArgumentsMissingException;
-import io.airlift.airline.ParseArgumentsUnexpectedException;
-import io.airlift.airline.ParseCommandMissingException;
-import io.airlift.airline.ParseCommandUnrecognizedException;
-import io.airlift.airline.ParseOptionConversionException;
-import io.airlift.airline.ParseOptionMissingException;
-import io.airlift.airline.ParseOptionMissingValueException;
-import org.apache.cassandra.tools.fqltool.commands.Dump;
-import org.apache.cassandra.tools.fqltool.commands.Replay;
-
-import static com.google.common.base.Throwables.getStackTraceAsString;
-import static com.google.common.collect.Lists.newArrayList;
-
-public class FullQueryLogTool
-{
-    public static void main(String... args)
-    {
-        List<Class<? extends Runnable>> commands = newArrayList(
-                Help.class,
-                Dump.class,
-                Replay.class
-        );
-
-        Cli.CliBuilder<Runnable> builder = Cli.builder("fqltool");
-
-        builder.withDescription("Manipulate the contents of full query log files")
-                 .withDefaultCommand(Help.class)
-                 .withCommands(commands);
-
-        Cli<Runnable> parser = builder.build();
-
-        int status = 0;
-        try
-        {
-            parser.parse(args).run();
-        } catch (IllegalArgumentException |
-                IllegalStateException |
-                ParseArgumentsMissingException |
-                ParseArgumentsUnexpectedException |
-                ParseOptionConversionException |
-                ParseOptionMissingException |
-                ParseOptionMissingValueException |
-                ParseCommandMissingException |
-                ParseCommandUnrecognizedException e)
-        {
-            badUse(e);
-            status = 1;
-        } catch (Throwable throwable)
-        {
-            err(Throwables.getRootCause(throwable));
-            status = 2;
-        }
-
-        System.exit(status);
-    }
-
-    private static void badUse(Exception e)
-    {
-        System.out.println("fqltool: " + e.getMessage());
-        System.out.println("See 'fqltool help' or 'fqltool help <command>'.");
-    }
-
-    private static void err(Throwable e)
-    {
-        System.err.println("error: " + e.getMessage());
-        System.err.println("-- StackTrace --");
-        System.err.println(getStackTraceAsString(e));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/DriverResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/DriverResultSet.java b/src/java/org/apache/cassandra/tools/fqltool/DriverResultSet.java
deleted file mode 100644
index 6c4ee45..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/DriverResultSet.java
+++ /dev/null
@@ -1,241 +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.tools.fqltool;
-
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.AbstractIterator;
-
-import com.datastax.driver.core.ColumnDefinitions;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Row;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-
-/**
- * Wraps a result set from the driver so that we can reuse the compare code when reading
- * up a result set produced by ResultStore.
- */
-public class DriverResultSet implements ResultHandler.ComparableResultSet
-{
-    private final ResultSet resultSet;
-    private final Throwable failureException;
-
-    public DriverResultSet(ResultSet resultSet)
-    {
-        this(resultSet, null);
-    }
-
-    private DriverResultSet(ResultSet res, Throwable failureException)
-    {
-        resultSet = res;
-        this.failureException = failureException;
-    }
-
-    public static DriverResultSet failed(Throwable ex)
-    {
-        return new DriverResultSet(null, ex);
-    }
-
-    public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
-    {
-        if (wasFailed())
-            return new DriverColumnDefinitions(null, true);
-
-        return new DriverColumnDefinitions(resultSet.getColumnDefinitions());
-    }
-
-    public boolean wasFailed()
-    {
-        return failureException != null;
-    }
-
-    public Throwable getFailureException()
-    {
-        return failureException;
-    }
-
-    public Iterator<ResultHandler.ComparableRow> iterator()
-    {
-        if (wasFailed())
-            return Collections.emptyListIterator();
-        return new AbstractIterator<ResultHandler.ComparableRow>()
-        {
-            Iterator<Row> iter = resultSet.iterator();
-            protected ResultHandler.ComparableRow computeNext()
-            {
-                if (iter.hasNext())
-                    return new DriverRow(iter.next());
-                return endOfData();
-            }
-        };
-    }
-
-    public static class DriverRow implements ResultHandler.ComparableRow
-    {
-        private final Row row;
-
-        public DriverRow(Row row)
-        {
-            this.row = row;
-        }
-
-        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
-        {
-            return new DriverColumnDefinitions(row.getColumnDefinitions());
-        }
-
-        public ByteBuffer getBytesUnsafe(int i)
-        {
-            return row.getBytesUnsafe(i);
-        }
-
-        @Override
-        public boolean equals(Object oo)
-        {
-            if (!(oo instanceof ResultHandler.ComparableRow))
-                return false;
-
-            ResultHandler.ComparableRow o = (ResultHandler.ComparableRow)oo;
-            if (getColumnDefinitions().size() != o.getColumnDefinitions().size())
-                return false;
-
-            for (int j = 0; j < getColumnDefinitions().size(); j++)
-            {
-                ByteBuffer b1 = getBytesUnsafe(j);
-                ByteBuffer b2 = o.getBytesUnsafe(j);
-
-                if (b1 != null && b2 != null && !b1.equals(b2))
-                {
-                    return false;
-                }
-                if (b1 == null && b2 != null || b2 == null && b1 != null)
-                {
-                    return false;
-                }
-            }
-            return true;
-        }
-
-        public String toString()
-        {
-            StringBuilder sb = new StringBuilder();
-            List<ResultHandler.ComparableDefinition> colDefs = getColumnDefinitions().asList();
-            for (int i = 0; i < getColumnDefinitions().size(); i++)
-            {
-                ByteBuffer bb = getBytesUnsafe(i);
-                String row = bb != null ? ByteBufferUtil.bytesToHex(bb) : "NULL";
-                sb.append(colDefs.get(i)).append(':').append(row).append(",");
-            }
-            return sb.toString();
-        }
-    }
-
-    public static class DriverColumnDefinitions implements ResultHandler.ComparableColumnDefinitions
-    {
-        private final ColumnDefinitions columnDefinitions;
-        private final boolean failed;
-
-        public DriverColumnDefinitions(ColumnDefinitions columnDefinitions)
-        {
-            this(columnDefinitions, false);
-        }
-
-        private DriverColumnDefinitions(ColumnDefinitions columnDefinitions, boolean failed)
-        {
-            this.columnDefinitions = columnDefinitions;
-            this.failed = failed;
-        }
-
-        public List<ResultHandler.ComparableDefinition> asList()
-        {
-            if (wasFailed())
-                return Collections.emptyList();
-            return columnDefinitions.asList().stream().map(DriverDefinition::new).collect(Collectors.toList());
-        }
-
-        public boolean wasFailed()
-        {
-            return failed;
-        }
-
-        public int size()
-        {
-            return columnDefinitions.size();
-        }
-
-        public Iterator<ResultHandler.ComparableDefinition> iterator()
-        {
-            return asList().iterator();
-        }
-
-        public boolean equals(Object oo)
-        {
-            if (!(oo instanceof ResultHandler.ComparableColumnDefinitions))
-                return false;
-
-            ResultHandler.ComparableColumnDefinitions o = (ResultHandler.ComparableColumnDefinitions)oo;
-            if (wasFailed() && o.wasFailed())
-                return true;
-
-            if (size() != o.size())
-                return false;
-
-            return asList().equals(o.asList());
-        }
-    }
-
-    public static class DriverDefinition implements ResultHandler.ComparableDefinition
-    {
-        private final ColumnDefinitions.Definition def;
-
-        public DriverDefinition(ColumnDefinitions.Definition def)
-        {
-            this.def = def;
-        }
-
-        public String getType()
-        {
-            return def.getType().toString();
-        }
-
-        public String getName()
-        {
-            return def.getName();
-        }
-
-        public boolean equals(Object oo)
-        {
-            if (!(oo instanceof ResultHandler.ComparableDefinition))
-                return false;
-
-            return def.equals(((DriverDefinition)oo).def);
-        }
-
-        public String toString()
-        {
-            return getName() + ':' + getType();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/FQLQuery.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/FQLQuery.java b/src/java/org/apache/cassandra/tools/fqltool/FQLQuery.java
deleted file mode 100644
index 6c0a6b9..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/FQLQuery.java
+++ /dev/null
@@ -1,278 +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.tools.fqltool;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import com.google.common.primitives.Longs;
-
-import com.datastax.driver.core.BatchStatement;
-import com.datastax.driver.core.ConsistencyLevel;
-import com.datastax.driver.core.SimpleStatement;
-import com.datastax.driver.core.Statement;
-import org.apache.cassandra.audit.FullQueryLogger;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.binlog.BinLog;
-
-public abstract class FQLQuery implements Comparable<FQLQuery>
-{
-    public final long queryStartTime;
-    public final QueryOptions queryOptions;
-    public final int protocolVersion;
-    public final String keyspace;
-    public final long generatedTimestamp;
-    private final int generatedNowInSeconds;
-
-    public FQLQuery(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds)
-    {
-        this.queryStartTime = queryStartTime;
-        this.queryOptions = queryOptions;
-        this.protocolVersion = protocolVersion;
-        this.keyspace = keyspace;
-        this.generatedTimestamp = generatedTimestamp;
-        this.generatedNowInSeconds = generatedNowInSeconds;
-    }
-
-    public abstract Statement toStatement();
-
-    /**
-     * used when storing the queries executed
-     */
-    public abstract BinLog.ReleaseableWriteMarshallable toMarshallable();
-
-    public QueryState queryState()
-    {
-        ClientState clientState = keyspace != null ? ClientState.forInternalCalls(keyspace) : ClientState.forInternalCalls();
-
-        return new QueryState(clientState, generatedTimestamp, generatedNowInSeconds);
-    }
-
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (!(o instanceof FQLQuery)) return false;
-        FQLQuery fqlQuery = (FQLQuery) o;
-        return queryStartTime == fqlQuery.queryStartTime &&
-               protocolVersion == fqlQuery.protocolVersion &&
-               generatedTimestamp == fqlQuery.generatedTimestamp &&
-               generatedNowInSeconds == fqlQuery.generatedNowInSeconds &&
-               Objects.equals(queryOptions.getValues(), fqlQuery.queryOptions.getValues()) &&
-               Objects.equals(keyspace, fqlQuery.keyspace);
-    }
-
-    public int hashCode()
-    {
-        return Objects.hash(queryStartTime, queryOptions, protocolVersion, keyspace, generatedTimestamp, generatedNowInSeconds);
-    }
-
-    public int compareTo(FQLQuery other)
-    {
-        int cmp = Longs.compare(queryStartTime, other.queryStartTime);
-        if (cmp != 0)
-            return cmp;
-        cmp = Longs.compare(generatedTimestamp, other.generatedTimestamp);
-        if (cmp != 0)
-            return cmp;
-
-        return Longs.compare(generatedNowInSeconds, other.generatedNowInSeconds);
-    }
-
-    public String toString()
-    {
-        return "FQLQuery{" +
-               "queryStartTime=" + queryStartTime +
-               ", protocolVersion=" + protocolVersion +
-               ", keyspace='" + keyspace + '\'' +
-               ", generatedTimestamp=" + generatedTimestamp +
-               ", generatedNowInSeconds=" + generatedNowInSeconds +
-               '}';
-    }
-
-    public static class Single extends FQLQuery
-    {
-        public final String query;
-        public final List<ByteBuffer> values;
-
-        public Single(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds, String queryString, List<ByteBuffer> values)
-        {
-            super(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds);
-            this.query = queryString;
-            this.values = values;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("%s%nQuery = %s, Values = %s",
-                                 super.toString(),
-                                 query,
-                                 values.stream().map(ByteBufferUtil::bytesToHex).collect(Collectors.joining(",")));
-        }
-
-        public Statement toStatement()
-        {
-            SimpleStatement ss = new SimpleStatement(query, values.toArray());
-            ss.setConsistencyLevel(ConsistencyLevel.valueOf(queryOptions.getConsistency().name()));
-            ss.setDefaultTimestamp(generatedTimestamp);
-            return ss;
-        }
-
-        public BinLog.ReleaseableWriteMarshallable toMarshallable()
-        {
-
-            return new FullQueryLogger.Query(query, queryOptions, queryState(), queryStartTime);
-        }
-
-        public int compareTo(FQLQuery other)
-        {
-            int cmp = super.compareTo(other);
-
-            if (cmp == 0)
-            {
-                if (other instanceof Batch)
-                    return -1;
-
-                Single singleQuery = (Single) other;
-
-                cmp = query.compareTo(singleQuery.query);
-                if (cmp == 0)
-                {
-                    if (values.size() != singleQuery.values.size())
-                        return values.size() - singleQuery.values.size();
-                    for (int i = 0; i < values.size(); i++)
-                    {
-                        cmp = values.get(i).compareTo(singleQuery.values.get(i));
-                        if (cmp != 0)
-                            return cmp;
-                    }
-                }
-            }
-            return cmp;
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (!(o instanceof Single)) return false;
-            if (!super.equals(o)) return false;
-            Single single = (Single) o;
-            return Objects.equals(query, single.query) &&
-                   Objects.equals(values, single.values);
-        }
-
-        public int hashCode()
-        {
-            return Objects.hash(super.hashCode(), query, values);
-        }
-    }
-
-    public static class Batch extends FQLQuery
-    {
-        public final BatchStatement.Type batchType;
-        public final List<Single> queries;
-
-        public Batch(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds, BatchStatement.Type batchType, List<String> queries, List<List<ByteBuffer>> values)
-        {
-            super(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds);
-            this.batchType = batchType;
-            this.queries = new ArrayList<>(queries.size());
-            for (int i = 0; i < queries.size(); i++)
-                this.queries.add(new Single(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds, queries.get(i), values.get(i)));
-        }
-
-        public Statement toStatement()
-        {
-            BatchStatement bs = new BatchStatement(batchType);
-
-            for (Single query : queries)
-            {
-                bs.add(new SimpleStatement(query.query, query.values.toArray()));
-            }
-            bs.setConsistencyLevel(ConsistencyLevel.valueOf(queryOptions.getConsistency().name()));
-            bs.setDefaultTimestamp(generatedTimestamp); // todo: set actual server side generated time
-            return bs;
-        }
-
-        public int compareTo(FQLQuery other)
-        {
-            int cmp = super.compareTo(other);
-
-            if (cmp == 0)
-            {
-                if (other instanceof Single)
-                    return 1;
-
-                Batch otherBatch = (Batch) other;
-                if (queries.size() != otherBatch.queries.size())
-                    return queries.size() - otherBatch.queries.size();
-                for (int i = 0; i < queries.size(); i++)
-                {
-                    cmp = queries.get(i).compareTo(otherBatch.queries.get(i));
-                    if (cmp != 0)
-                        return cmp;
-                }
-            }
-            return cmp;
-        }
-
-        public BinLog.ReleaseableWriteMarshallable toMarshallable()
-        {
-            List<String> queryStrings = new ArrayList<>();
-            List<List<ByteBuffer>> values = new ArrayList<>();
-            for (Single q : queries)
-            {
-                queryStrings.add(q.query);
-                values.add(q.values);
-            }
-            return new FullQueryLogger.Batch(org.apache.cassandra.cql3.statements.BatchStatement.Type.valueOf(batchType.name()), queryStrings, values, queryOptions, queryState(), queryStartTime);
-        }
-
-        public String toString()
-        {
-            StringBuilder sb = new StringBuilder(super.toString()).append("\nbatch: ").append(batchType).append('\n');
-            for (Single q : queries)
-                sb.append(q.toString()).append('\n');
-            sb.append("end batch");
-            return sb.toString();
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (!(o instanceof Batch)) return false;
-            if (!super.equals(o)) return false;
-            Batch batch = (Batch) o;
-            return batchType == batch.batchType &&
-                   Objects.equals(queries, batch.queries);
-        }
-
-        public int hashCode()
-        {
-            return Objects.hash(super.hashCode(), batchType, queries);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/FQLQueryIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/FQLQueryIterator.java b/src/java/org/apache/cassandra/tools/fqltool/FQLQueryIterator.java
deleted file mode 100644
index 390a52e..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/FQLQueryIterator.java
+++ /dev/null
@@ -1,72 +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.tools.fqltool;
-
-import java.util.PriorityQueue;
-
-import net.openhft.chronicle.queue.ExcerptTailer;
-import org.apache.cassandra.utils.AbstractIterator;
-
-public class FQLQueryIterator extends AbstractIterator<FQLQuery>
-{
-    // use a priority queue to be able to sort the head of the query logs in memory
-    private final PriorityQueue<FQLQuery> pq;
-    private final ExcerptTailer tailer;
-    private final FQLQueryReader reader;
-
-    /**
-     * Create an iterator over the FQLQueries in tailer
-     *
-     * Reads up to readAhead queries in to memory to be able to sort them (the files are mostly sorted already)
-     */
-    public FQLQueryIterator(ExcerptTailer tailer, int readAhead)
-    {
-        assert readAhead > 0 : "readAhead needs to be > 0";
-        reader = new FQLQueryReader();
-        this.tailer = tailer;
-        pq = new PriorityQueue<>(readAhead);
-        for (int i = 0; i < readAhead; i++)
-        {
-            FQLQuery next = readNext();
-            if (next != null)
-                pq.add(next);
-            else
-                break;
-        }
-    }
-
-    protected FQLQuery computeNext()
-    {
-        FQLQuery q = pq.poll();
-        if (q == null)
-            return endOfData();
-        FQLQuery next = readNext();
-        if (next != null)
-            pq.add(next);
-        return q;
-    }
-
-    private FQLQuery readNext()
-    {
-        if (tailer.readDocument(reader))
-            return reader.getQuery();
-        return null;
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/FQLQueryReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/FQLQueryReader.java b/src/java/org/apache/cassandra/tools/fqltool/FQLQueryReader.java
deleted file mode 100644
index af77c59..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/FQLQueryReader.java
+++ /dev/null
@@ -1,116 +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.tools.fqltool;
-
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import com.datastax.driver.core.BatchStatement;
-import io.netty.buffer.Unpooled;
-import net.openhft.chronicle.core.io.IORuntimeException;
-import net.openhft.chronicle.wire.ReadMarshallable;
-import net.openhft.chronicle.wire.ValueIn;
-import net.openhft.chronicle.wire.WireIn;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.transport.ProtocolVersion;
-
-import static org.apache.cassandra.audit.FullQueryLogger.GENERATED_NOW_IN_SECONDS;
-import static org.apache.cassandra.audit.FullQueryLogger.GENERATED_TIMESTAMP;
-import static org.apache.cassandra.audit.FullQueryLogger.KEYSPACE;
-import static org.apache.cassandra.audit.FullQueryLogger.PROTOCOL_VERSION;
-import static org.apache.cassandra.audit.FullQueryLogger.QUERY_OPTIONS;
-import static org.apache.cassandra.audit.FullQueryLogger.QUERY_START_TIME;
-import static org.apache.cassandra.audit.FullQueryLogger.TYPE;
-import static org.apache.cassandra.audit.FullQueryLogger.VERSION;
-import static org.apache.cassandra.audit.FullQueryLogger.BATCH;
-import static org.apache.cassandra.audit.FullQueryLogger.BATCH_TYPE;
-import static org.apache.cassandra.audit.FullQueryLogger.QUERIES;
-import static org.apache.cassandra.audit.FullQueryLogger.QUERY;
-import static org.apache.cassandra.audit.FullQueryLogger.SINGLE_QUERY;
-import static org.apache.cassandra.audit.FullQueryLogger.VALUES;
-
-public class FQLQueryReader implements ReadMarshallable
-{
-    private FQLQuery query;
-
-    public void readMarshallable(WireIn wireIn) throws IORuntimeException
-    {
-        int currentVersion = wireIn.read(VERSION).int16();
-        String type = wireIn.read(TYPE).text();
-        long queryStartTime = wireIn.read(QUERY_START_TIME).int64();
-        int protocolVersion = wireIn.read(PROTOCOL_VERSION).int32();
-        QueryOptions queryOptions = QueryOptions.codec.decode(Unpooled.wrappedBuffer(wireIn.read(QUERY_OPTIONS).bytes()), ProtocolVersion.decode(protocolVersion));
-        long generatedTimestamp = wireIn.read(GENERATED_TIMESTAMP).int64();
-        int generatedNowInSeconds = wireIn.read(GENERATED_NOW_IN_SECONDS).int32();
-        String keyspace = wireIn.read(KEYSPACE).text();
-
-        switch (type)
-        {
-            case SINGLE_QUERY:
-                String queryString = wireIn.read(QUERY).text();
-                query = new FQLQuery.Single(keyspace,
-                                            protocolVersion,
-                                            queryOptions,
-                                            queryStartTime,
-                                            generatedTimestamp,
-                                            generatedNowInSeconds,
-                                            queryString,
-                                            queryOptions.getValues());
-                break;
-            case BATCH:
-                BatchStatement.Type batchType = BatchStatement.Type.valueOf(wireIn.read(BATCH_TYPE).text());
-                ValueIn in = wireIn.read(QUERIES);
-                int queryCount = in.int32();
-
-                List<String> queries = new ArrayList<>(queryCount);
-                for (int i = 0; i < queryCount; i++)
-                    queries.add(in.text());
-                in = wireIn.read(VALUES);
-                int valueCount = in.int32();
-                List<List<ByteBuffer>> values = new ArrayList<>(valueCount);
-                for (int ii = 0; ii < valueCount; ii++)
-                {
-                    List<ByteBuffer> subValues = new ArrayList<>();
-                    values.add(subValues);
-                    int numSubValues = in.int32();
-                    for (int zz = 0; zz < numSubValues; zz++)
-                        subValues.add(ByteBuffer.wrap(in.bytes()));
-                }
-                query = new FQLQuery.Batch(keyspace,
-                                           protocolVersion,
-                                           queryOptions,
-                                           queryStartTime,
-                                           generatedTimestamp,
-                                           generatedNowInSeconds,
-                                           batchType,
-                                           queries,
-                                           values);
-                break;
-            default:
-                throw new RuntimeException("Unknown type: " + type);
-        }
-    }
-
-    public FQLQuery getQuery()
-    {
-        return query;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/QueryReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/QueryReplayer.java b/src/java/org/apache/cassandra/tools/fqltool/QueryReplayer.java
deleted file mode 100644
index 0c8382f..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/QueryReplayer.java
+++ /dev/null
@@ -1,167 +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.tools.fqltool;
-
-import java.io.Closeable;
-import java.io.File;
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import javax.annotation.Nullable;
-
-import com.google.common.util.concurrent.FluentFuture;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
-
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.Timer;
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.ConsistencyLevel;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Session;
-import com.datastax.driver.core.Statement;
-import org.apache.cassandra.utils.FBUtilities;
-
-public class QueryReplayer implements Closeable
-{
-    private static final int PRINT_RATE = 5000;
-    private final ExecutorService es = Executors.newFixedThreadPool(1);
-    private final Iterator<List<FQLQuery>> queryIterator;
-    private final List<Cluster> targetClusters;
-    private final List<Predicate<FQLQuery>> filters;
-    private final List<Session> sessions;
-    private final ResultHandler resultHandler;
-    private final MetricRegistry metrics = new MetricRegistry();
-    private final boolean debug;
-    private final PrintStream out;
-
-    public QueryReplayer(Iterator<List<FQLQuery>> queryIterator,
-                         List<String> targetHosts,
-                         List<File> resultPaths,
-                         List<Predicate<FQLQuery>> filters,
-                         PrintStream out,
-                         String queryFilePathString,
-                         boolean debug)
-    {
-        this.queryIterator = queryIterator;
-        targetClusters = targetHosts.stream().map(h -> Cluster.builder().addContactPoint(h).build()).collect(Collectors.toList());
-        this.filters = filters;
-        sessions = targetClusters.stream().map(Cluster::connect).collect(Collectors.toList());
-        File queryFilePath = queryFilePathString != null ? new File(queryFilePathString) : null;
-        resultHandler = new ResultHandler(targetHosts, resultPaths, queryFilePath);
-        this.debug = debug;
-        this.out = out;
-    }
-
-    public void replay()
-    {
-        while (queryIterator.hasNext())
-        {
-            List<FQLQuery> queries = queryIterator.next();
-            for (FQLQuery query : queries)
-            {
-                if (filters.stream().anyMatch(f -> !f.test(query)))
-                    continue;
-                try (Timer.Context ctx = metrics.timer("queries").time())
-                {
-                    List<ListenableFuture<ResultHandler.ComparableResultSet>> results = new ArrayList<>(sessions.size());
-                    Statement statement = query.toStatement();
-                    for (Session session : sessions)
-                    {
-                        try
-                        {
-                            if (query.keyspace != null && !query.keyspace.equals(session.getLoggedKeyspace()))
-                            {
-                                if (debug)
-                                    out.printf("Switching keyspace from %s to %s%n", session.getLoggedKeyspace(), query.keyspace);
-                                session.execute("USE " + query.keyspace);
-                            }
-                        }
-                        catch (Throwable t)
-                        {
-                            out.printf("USE %s failed: %s%n", query.keyspace, t.getMessage());
-                        }
-                        if (debug)
-                        {
-                            out.println("Executing query:");
-                            out.println(query);
-                        }
-                        ListenableFuture<ResultSet> future = session.executeAsync(statement);
-                        results.add(handleErrors(future));
-                    }
-
-                    ListenableFuture<List<ResultHandler.ComparableResultSet>> resultList = Futures.allAsList(results);
-
-                    Futures.addCallback(resultList, new FutureCallback<List<ResultHandler.ComparableResultSet>>()
-                    {
-                        public void onSuccess(@Nullable List<ResultHandler.ComparableResultSet> resultSets)
-                        {
-                            // note that the order of resultSets is signifcant here - resultSets.get(x) should
-                            // be the result from a query against targetHosts.get(x)
-                            resultHandler.handleResults(query, resultSets);
-                        }
-
-                        public void onFailure(Throwable throwable)
-                        {
-                            throw new AssertionError("Errors should be handled in FQLQuery.execute", throwable);
-                        }
-                    }, es);
-
-                    FBUtilities.waitOnFuture(resultList);
-                }
-                catch (Throwable t)
-                {
-                    out.printf("QUERY %s got exception: %s", query, t.getMessage());
-                }
-
-                Timer timer = metrics.timer("queries");
-                if (timer.getCount() % PRINT_RATE == 0)
-                    out.printf("%d queries, rate = %.2f%n", timer.getCount(), timer.getOneMinuteRate());
-            }
-        }
-    }
-
-    /**
-     * Make sure we catch any query errors
-     *
-     * On error, this creates a failed ComparableResultSet with the exception set to be able to store
-     * this fact in the result file and handle comparison of failed result sets.
-     */
-    private static ListenableFuture<ResultHandler.ComparableResultSet> handleErrors(ListenableFuture<ResultSet> result)
-    {
-        FluentFuture<ResultHandler.ComparableResultSet> fluentFuture = FluentFuture.from(result)
-                                                                                   .transform(DriverResultSet::new, MoreExecutors.directExecutor());
-        return fluentFuture.catching(Throwable.class, DriverResultSet::failed, MoreExecutors.directExecutor());
-    }
-
-    public void close()
-    {
-        sessions.forEach(Session::close);
-        targetClusters.forEach(Cluster::close);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/ResultComparator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/ResultComparator.java b/src/java/org/apache/cassandra/tools/fqltool/ResultComparator.java
deleted file mode 100644
index 4bbaf7a..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/ResultComparator.java
+++ /dev/null
@@ -1,116 +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.tools.fqltool;
-
-
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Streams;
-
-public class ResultComparator
-{
-    /**
-     * Compares the rows in rows
-     * the row at position x in rows will have come from host at position x in targetHosts
-     */
-    public boolean compareRows(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableRow> rows)
-    {
-        if (rows.size() < 2 || rows.stream().allMatch(Objects::isNull))
-            return true;
-
-        if (rows.stream().anyMatch(Objects::isNull))
-        {
-            handleMismatch(targetHosts, query, rows);
-            return false;
-        }
-
-        ResultHandler.ComparableRow ref = rows.get(0);
-        boolean equal = true;
-        for (int i = 1; i < rows.size(); i++)
-        {
-            ResultHandler.ComparableRow compare = rows.get(i);
-            if (!ref.equals(compare))
-                equal = false;
-        }
-        if (!equal)
-            handleMismatch(targetHosts, query, rows);
-        return equal;
-    }
-
-    /**
-     * Compares the column definitions
-     *
-     * the column definitions at position x in cds will have come from host at position x in targetHosts
-     */
-    public boolean compareColumnDefinitions(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
-    {
-        if (cds.size() < 2)
-            return true;
-
-        boolean equal = true;
-        List<ResultHandler.ComparableDefinition> refDefs = cds.get(0).asList();
-        for (int i = 1; i < cds.size(); i++)
-        {
-            List<ResultHandler.ComparableDefinition> toCompare = cds.get(i).asList();
-            if (!refDefs.equals(toCompare))
-                equal = false;
-        }
-        if (!equal)
-            handleColumnDefMismatch(targetHosts, query, cds);
-        return equal;
-    }
-
-    private void handleMismatch(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableRow> rows)
-    {
-        System.out.println("MISMATCH:");
-        System.out.println("Query = " + query);
-        System.out.println("Results:");
-        System.out.println(Streams.zip(rows.stream(), targetHosts.stream(), (r, host) -> String.format("%s: %s%n", host, r == null ? "null" : r)).collect(Collectors.joining()));
-    }
-
-    private void handleColumnDefMismatch(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
-    {
-        System.out.println("COLUMN DEFINITION MISMATCH:");
-        System.out.println("Query = " + query);
-        System.out.println("Results: ");
-        System.out.println(Streams.zip(cds.stream(), targetHosts.stream(), (cd, host) -> String.format("%s: %s%n", host, columnDefinitionsString(cd))).collect(Collectors.joining()));
-    }
-
-    private String columnDefinitionsString(ResultHandler.ComparableColumnDefinitions cd)
-    {
-        StringBuilder sb = new StringBuilder();
-        if (cd == null)
-            sb.append("NULL");
-        else if (cd.wasFailed())
-            sb.append("FAILED");
-        else
-        {
-            for (ResultHandler.ComparableDefinition def : cd)
-            {
-                sb.append(def.toString());
-            }
-        }
-        return sb.toString();
-    }
-
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/ResultHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/ResultHandler.java b/src/java/org/apache/cassandra/tools/fqltool/ResultHandler.java
deleted file mode 100644
index c769231..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/ResultHandler.java
+++ /dev/null
@@ -1,124 +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.tools.fqltool;
-
-import java.io.File;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-import com.google.common.annotations.VisibleForTesting;
-
-public class ResultHandler
-{
-    private final ResultStore resultStore;
-    private final ResultComparator resultComparator;
-    private final List<String> targetHosts;
-
-    public ResultHandler(List<String> targetHosts, List<File> resultPaths, File queryFilePath)
-    {
-        this.targetHosts = targetHosts;
-        resultStore = resultPaths != null ? new ResultStore(resultPaths, queryFilePath) : null;
-        resultComparator = new ResultComparator();
-    }
-
-    /**
-     * Since we can't iterate a ResultSet more than once, and we don't want to keep the entire result set in memory
-     * we feed the rows one-by-one to resultComparator and resultStore.
-     *
-     * results.get(x) should be the results from executing query against targetHosts.get(x)
-     */
-    public void handleResults(FQLQuery query, List<ComparableResultSet> results)
-    {
-        for (int i = 0; i < targetHosts.size(); i++)
-        {
-            if (results.get(i).wasFailed())
-            {
-                System.out.println("Query against "+targetHosts.get(i)+" failure:");
-                System.out.println(query);
-                System.out.println("Message: "+results.get(i).getFailureException().getMessage());
-            }
-        }
-
-        List<ComparableColumnDefinitions> columnDefinitions = results.stream().map(ComparableResultSet::getColumnDefinitions).collect(Collectors.toList());
-        resultComparator.compareColumnDefinitions(targetHosts, query, columnDefinitions);
-        if (resultStore != null)
-            resultStore.storeColumnDefinitions(query, columnDefinitions);
-        List<Iterator<ComparableRow>> iters = results.stream().map(Iterable::iterator).collect(Collectors.toList());
-
-        while (true)
-        {
-            List<ComparableRow> rows = rows(iters);
-            resultComparator.compareRows(targetHosts, query, rows);
-            if (resultStore != null)
-                resultStore.storeRows(rows);
-            // all rows being null marks end of all resultsets, we need to call compareRows
-            // and storeRows once with everything null to mark that fact
-            if (rows.stream().allMatch(Objects::isNull))
-                return;
-        }
-    }
-
-    /**
-     * Get the first row from each of the iterators, if the iterator has run out, null will mark that in the list
-     */
-    @VisibleForTesting
-    public static List<ComparableRow> rows(List<Iterator<ComparableRow>> iters)
-    {
-        List<ComparableRow> rows = new ArrayList<>(iters.size());
-        for (Iterator<ComparableRow> iter : iters)
-        {
-            if (iter.hasNext())
-                rows.add(iter.next());
-            else
-                rows.add(null);
-        }
-        return rows;
-    }
-
-    public interface ComparableResultSet extends Iterable<ComparableRow>
-    {
-        public ComparableColumnDefinitions getColumnDefinitions();
-        public boolean wasFailed();
-        public Throwable getFailureException();
-    }
-
-    public interface ComparableColumnDefinitions extends Iterable<ComparableDefinition>
-    {
-        public List<ComparableDefinition> asList();
-        public boolean wasFailed();
-        public int size();
-    }
-
-    public interface ComparableDefinition
-    {
-        public String getType();
-        public String getName();
-    }
-
-    public interface ComparableRow
-    {
-        public ByteBuffer getBytesUnsafe(int i);
-        public ComparableColumnDefinitions getColumnDefinitions();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/ResultStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/ResultStore.java b/src/java/org/apache/cassandra/tools/fqltool/ResultStore.java
deleted file mode 100644
index 6d6aaac..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/ResultStore.java
+++ /dev/null
@@ -1,142 +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.tools.fqltool;
-
-
-import java.io.File;
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import net.openhft.chronicle.bytes.BytesStore;
-import net.openhft.chronicle.core.io.Closeable;
-import net.openhft.chronicle.queue.ChronicleQueue;
-import net.openhft.chronicle.queue.ChronicleQueueBuilder;
-import net.openhft.chronicle.queue.ExcerptAppender;
-import net.openhft.chronicle.wire.ValueOut;
-import org.apache.cassandra.utils.binlog.BinLog;
-
-/**
- * see FQLReplayTest#readResultFile for how to read files produced by this class
- */
-public class ResultStore
-{
-    private final List<ChronicleQueue> queues;
-    private final List<ExcerptAppender> appenders;
-    private final ChronicleQueue queryStoreQueue;
-    private final ExcerptAppender queryStoreAppender;
-    private final Set<Integer> finishedHosts = new HashSet<>();
-
-    public ResultStore(List<File> resultPaths, File queryFilePath)
-    {
-        queues = resultPaths.stream().map(path -> ChronicleQueueBuilder.single(path).build()).collect(Collectors.toList());
-        appenders = queues.stream().map(ChronicleQueue::acquireAppender).collect(Collectors.toList());
-        queryStoreQueue = queryFilePath != null ? ChronicleQueueBuilder.single(queryFilePath).build() : null;
-        queryStoreAppender = queryStoreQueue != null ? queryStoreQueue.acquireAppender() : null;
-    }
-
-    /**
-     * Store the column definitions in cds
-     *
-     * the ColumnDefinitions at position x will get stored by the appender at position x
-     *
-     * Calling this method indicates that we are starting a new result set from a query, it must be called before
-     * calling storeRows.
-     *
-     */
-    public void storeColumnDefinitions(FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
-    {
-        finishedHosts.clear();
-        if (queryStoreAppender != null)
-        {
-            BinLog.ReleaseableWriteMarshallable writeMarshallableQuery = query.toMarshallable();
-            queryStoreAppender.writeDocument(writeMarshallableQuery);
-            writeMarshallableQuery.release();
-        }
-        for (int i = 0; i < cds.size(); i++)
-        {
-            ResultHandler.ComparableColumnDefinitions cd = cds.get(i);
-            appenders.get(i).writeDocument(wire ->
-                                           {
-                                               if (!cd.wasFailed())
-                                               {
-                                                   wire.write("type").text("column_definitions");
-                                                   wire.write("column_count").int32(cd.size());
-                                                   for (ResultHandler.ComparableDefinition d : cd.asList())
-                                                   {
-                                                       ValueOut vo = wire.write("column_definition");
-                                                       vo.text(d.getName());
-                                                       vo.text(d.getType());
-                                                   }
-                                               }
-                                               else
-                                               {
-                                                   wire.write("type").text("query_failed");
-                                               }
-                                           });
-        }
-    }
-
-    /**
-     * Store rows
-     *
-     * the row at position x will get stored by appender at position x
-     *
-     * Before calling this for a new result set, storeColumnDefinitions must be called.
-     */
-    public void storeRows(List<ResultHandler.ComparableRow> rows)
-    {
-        for (int i = 0; i < rows.size(); i++)
-        {
-            ResultHandler.ComparableRow row = rows.get(i);
-            if (row == null && !finishedHosts.contains(i))
-            {
-                appenders.get(i).writeDocument(wire -> wire.write("type").text("end_resultset"));
-                finishedHosts.add(i);
-            }
-            else if (row != null)
-            {
-                appenders.get(i).writeDocument(wire ->
-                                               {
-                                                   {
-                                                       wire.write("type").text("row");
-                                                       wire.write("row_column_count").int32(row.getColumnDefinitions().size());
-                                                       for (int jj = 0; jj < row.getColumnDefinitions().size(); jj++)
-                                                       {
-                                                           ByteBuffer bb = row.getBytesUnsafe(jj);
-                                                           if (bb != null)
-                                                               wire.write("column").bytes(BytesStore.wrap(bb));
-                                                           else
-                                                               wire.write("column").bytes("NULL".getBytes());
-                                                       }
-                                                   }
-                                               });
-            }
-        }
-    }
-
-    public void close()
-    {
-        queues.forEach(Closeable::close);
-        if (queryStoreQueue != null)
-            queryStoreQueue.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/commands/Dump.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/commands/Dump.java b/src/java/org/apache/cassandra/tools/fqltool/commands/Dump.java
deleted file mode 100644
index 5c23d3e..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/commands/Dump.java
+++ /dev/null
@@ -1,325 +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.tools.fqltool.commands;
-
-import java.io.File;
-import java.nio.BufferUnderflowException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import io.airlift.airline.Arguments;
-import io.airlift.airline.Command;
-import io.airlift.airline.Option;
-import io.netty.buffer.Unpooled;
-import net.openhft.chronicle.bytes.Bytes;
-import net.openhft.chronicle.queue.ChronicleQueue;
-import net.openhft.chronicle.queue.ChronicleQueueBuilder;
-import net.openhft.chronicle.queue.ExcerptTailer;
-import net.openhft.chronicle.queue.RollCycles;
-import net.openhft.chronicle.threads.Pauser;
-import net.openhft.chronicle.wire.ReadMarshallable;
-import net.openhft.chronicle.wire.ValueIn;
-import net.openhft.chronicle.wire.WireIn;
-import org.apache.cassandra.audit.FullQueryLogger;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.transport.ProtocolVersion;
-
-/**
- * Dump the contents of a list of paths containing full query logs
- */
-@Command(name = "dump", description = "Dump the contents of a full query log")
-public class Dump implements Runnable
-{
-    static final char[] HEXI_DECIMAL = "0123456789ABCDEF".toCharArray();
-
-    @Arguments(usage = "<path1> [<path2>...<pathN>]", description = "Path containing the full query logs to dump.", required = true)
-    private List<String> arguments = new ArrayList<>();
-
-    @Option(title = "roll_cycle", name = {"--roll-cycle"}, description = "How often to roll the log file was rolled. May be necessary for Chronicle to correctly parse file names. (MINUTELY, HOURLY, DAILY). Default HOURLY.")
-    private String rollCycle = "HOURLY";
-
-    @Option(title = "follow", name = {"--follow"}, description = "Upon reacahing the end of the log continue indefinitely waiting for more records")
-    private boolean follow = false;
-
-    @Override
-    public void run()
-    {
-        dump(arguments, rollCycle, follow);
-    }
-
-    public static void dump(List<String> arguments, String rollCycle, boolean follow)
-    {
-        StringBuilder sb = new StringBuilder();
-        ReadMarshallable reader = wireIn ->
-        {
-            sb.setLength(0);
-
-            int version = wireIn.read(FullQueryLogger.VERSION).int16();
-            if (version != FullQueryLogger.CURRENT_VERSION)
-                throw new UnsupportedOperationException("Full query log of unexpected version " + version + " encountered");
-
-            String type = wireIn.read(FullQueryLogger.TYPE).text();
-            sb.append("Type: ")
-              .append(type)
-              .append(System.lineSeparator());
-
-            long queryStartTime = wireIn.read(FullQueryLogger.QUERY_START_TIME).int64();
-            sb.append("Query start time: ")
-              .append(queryStartTime)
-              .append(System.lineSeparator());
-
-            int protocolVersion = wireIn.read(FullQueryLogger.PROTOCOL_VERSION).int32();
-            sb.append("Protocol version: ")
-              .append(protocolVersion)
-              .append(System.lineSeparator());
-
-            QueryOptions options =
-                QueryOptions.codec.decode(Unpooled.wrappedBuffer(wireIn.read(FullQueryLogger.QUERY_OPTIONS).bytes()),
-                                          ProtocolVersion.decode(protocolVersion));
-
-            long generatedTimestamp = wireIn.read(FullQueryLogger.GENERATED_TIMESTAMP).int64();
-            sb.append("Generated timestamp:")
-              .append(generatedTimestamp)
-              .append(System.lineSeparator());
-
-            int generatedNowInSeconds = wireIn.read(FullQueryLogger.GENERATED_NOW_IN_SECONDS).int32();
-            sb.append("Generated nowInSeconds:")
-              .append(generatedNowInSeconds)
-              .append(System.lineSeparator());
-
-            switch (type)
-            {
-                case (FullQueryLogger.SINGLE_QUERY):
-                    dumpQuery(options, wireIn, sb);
-                    break;
-
-                case (FullQueryLogger.BATCH):
-                    dumpBatch(options, wireIn, sb);
-                    break;
-
-                default:
-                    throw new UnsupportedOperationException("Log entry of unsupported type " + type);
-            }
-
-            System.out.print(sb.toString());
-            System.out.flush();
-        };
-
-        //Backoff strategy for spinning on the queue, not aggressive at all as this doesn't need to be low latency
-        Pauser pauser = Pauser.millis(100);
-        List<ChronicleQueue> queues = arguments.stream().distinct().map(path -> ChronicleQueueBuilder.single(new File(path)).readOnly(true).rollCycle(RollCycles.valueOf(rollCycle)).build()).collect(Collectors.toList());
-        List<ExcerptTailer> tailers = queues.stream().map(ChronicleQueue::createTailer).collect(Collectors.toList());
-        boolean hadWork = true;
-        while (hadWork)
-        {
-            hadWork = false;
-            for (ExcerptTailer tailer : tailers)
-            {
-                while (tailer.readDocument(reader))
-                {
-                    hadWork = true;
-                }
-            }
-
-            if (follow)
-            {
-                if (!hadWork)
-                {
-                    //Chronicle queue doesn't support blocking so use this backoff strategy
-                    pauser.pause();
-                }
-                //Don't terminate the loop even if there wasn't work
-                hadWork = true;
-            }
-        }
-    }
-
-    private static void dumpQuery(QueryOptions options, WireIn wireIn, StringBuilder sb)
-    {
-        sb.append("Query: ")
-          .append(wireIn.read(FullQueryLogger.QUERY).text())
-          .append(System.lineSeparator());
-
-        List<ByteBuffer> values = options.getValues() != null
-                                ? options.getValues()
-                                : Collections.emptyList();
-
-        sb.append("Values: ")
-          .append(System.lineSeparator());
-        appendValuesToStringBuilder(values, sb);
-        sb.append(System.lineSeparator());
-    }
-
-    private static void dumpBatch(QueryOptions options, WireIn wireIn, StringBuilder sb)
-    {
-        sb.append("Batch type: ")
-          .append(wireIn.read(FullQueryLogger.BATCH_TYPE).text())
-          .append(System.lineSeparator());
-
-        ValueIn in = wireIn.read(FullQueryLogger.QUERIES);
-        int numQueries = in.int32();
-        List<String> queries = new ArrayList<>(numQueries);
-        for (int i = 0; i < numQueries; i++)
-            queries.add(in.text());
-
-        in = wireIn.read(FullQueryLogger.VALUES);
-        int numValues = in.int32();
-
-        for (int i = 0; i < numValues; i++)
-        {
-            int numSubValues = in.int32();
-            List<ByteBuffer> subValues = new ArrayList<>(numSubValues);
-            for (int j = 0; j < numSubValues; j++)
-                subValues.add(ByteBuffer.wrap(in.bytes()));
-
-            sb.append("Query: ")
-              .append(queries.get(i))
-              .append(System.lineSeparator());
-
-            sb.append("Values: ")
-              .append(System.lineSeparator());
-            appendValuesToStringBuilder(subValues, sb);
-        }
-
-        sb.append(System.lineSeparator());
-    }
-
-    private static void appendValuesToStringBuilder(List<ByteBuffer> values, StringBuilder sb)
-    {
-        boolean first = true;
-        for (ByteBuffer value : values)
-        {
-            Bytes bytes = Bytes.wrapForRead(value);
-            long maxLength2 = Math.min(1024, bytes.readLimit() - bytes.readPosition());
-            toHexString(bytes, bytes.readPosition(), maxLength2, sb);
-            if (maxLength2 < bytes.readLimit() - bytes.readPosition())
-            {
-                sb.append("... truncated").append(System.lineSeparator());
-            }
-
-            if (first)
-            {
-                first = false;
-            }
-            else
-            {
-                sb.append("-----").append(System.lineSeparator());
-            }
-        }
-    }
-
-    //This is from net.openhft.chronicle.bytes, need to pass in the StringBuilder so had to copy
-    /*
-     * Copyright 2016 higherfrequencytrading.com
-     *
-     * Licensed 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.
-     */
-    /**
-     * display the hex data of {@link Bytes} from the position() to the limit()
-     *
-     * @param bytes the buffer you wish to toString()
-     * @return hex representation of the buffer, from example [0D ,OA, FF]
-     */
-    public static String toHexString(final Bytes bytes, long offset, long len, StringBuilder builder)
-    throws BufferUnderflowException
-    {
-        if (len == 0)
-            return "";
-
-        int width = 16;
-        int[] lastLine = new int[width];
-        String sep = "";
-        long position = bytes.readPosition();
-        long limit = bytes.readLimit();
-
-        try {
-            bytes.readPositionRemaining(offset, len);
-
-            long start = offset / width * width;
-            long end = (offset + len + width - 1) / width * width;
-            for (long i = start; i < end; i += width) {
-                // check for duplicate rows
-                if (i + width < end) {
-                    boolean same = true;
-
-                    for (int j = 0; j < width && i + j < offset + len; j++) {
-                        int ch = bytes.readUnsignedByte(i + j);
-                        same &= (ch == lastLine[j]);
-                        lastLine[j] = ch;
-                    }
-                    if (i > start && same) {
-                        sep = "........\n";
-                        continue;
-                    }
-                }
-                builder.append(sep);
-                sep = "";
-                String str = Long.toHexString(i);
-                for (int j = str.length(); j < 8; j++)
-                    builder.append('0');
-                builder.append(str);
-                for (int j = 0; j < width; j++) {
-                    if (j == width / 2)
-                        builder.append(' ');
-                    if (i + j < offset || i + j >= offset + len) {
-                        builder.append("   ");
-
-                    } else {
-                        builder.append(' ');
-                        int ch = bytes.readUnsignedByte(i + j);
-                        builder.append(HEXI_DECIMAL[ch >> 4]);
-                        builder.append(HEXI_DECIMAL[ch & 15]);
-                    }
-                }
-                builder.append(' ');
-                for (int j = 0; j < width; j++) {
-                    if (j == width / 2)
-                        builder.append(' ');
-                    if (i + j < offset || i + j >= offset + len) {
-                        builder.append(' ');
-
-                    } else {
-                        int ch = bytes.readUnsignedByte(i + j);
-                        if (ch < ' ' || ch > 126)
-                            ch = '\u00B7';
-                        builder.append((char) ch);
-                    }
-                }
-                builder.append("\n");
-            }
-            return builder.toString();
-        } finally {
-            bytes.readLimit(limit);
-            bytes.readPosition(position);
-        }
-    }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[6/7] cassandra git commit: Add fqltool compare

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/src/java/org/apache/cassandra/tools/fqltool/commands/Replay.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/commands/Replay.java b/src/java/org/apache/cassandra/tools/fqltool/commands/Replay.java
deleted file mode 100644
index 043ead8..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/commands/Replay.java
+++ /dev/null
@@ -1,148 +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.tools.fqltool.commands;
-
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import io.airlift.airline.Arguments;
-import io.airlift.airline.Command;
-import io.airlift.airline.Option;
-import net.openhft.chronicle.core.io.Closeable;
-import net.openhft.chronicle.queue.ChronicleQueue;
-import net.openhft.chronicle.queue.ChronicleQueueBuilder;
-
-import org.apache.cassandra.tools.fqltool.FQLQuery;
-import org.apache.cassandra.tools.fqltool.FQLQueryIterator;
-import org.apache.cassandra.tools.fqltool.QueryReplayer;
-import org.apache.cassandra.utils.AbstractIterator;
-import org.apache.cassandra.utils.MergeIterator;
-
-/**
- * replay the contents of a list of paths containing full query logs
- */
-@Command(name = "replay", description = "Replay full query logs")
-public class Replay implements Runnable
-{
-    @Arguments(usage = "<path1> [<path2>...<pathN>]", description = "Paths containing the full query logs to replay.", required = true)
-    private List<String> arguments = new ArrayList<>();
-
-    @Option(title = "target", name = {"--target"}, description = "Hosts to replay the logs to, can be repeated to replay to more hosts.")
-    private List<String> targetHosts;
-
-    @Option(title = "results", name = { "--results"}, description = "Where to store the results of the queries, this should be a directory. Leave this option out to avoid storing results.")
-    private String resultPath;
-
-    @Option(title = "keyspace", name = { "--keyspace"}, description = "Only replay queries against this keyspace and queries without keyspace set.")
-    private String keyspace;
-
-    @Option(title = "debug", name = {"--debug"}, description = "Debug mode, print all queries executed.")
-    private boolean debug;
-
-    @Option(title = "store_queries", name = {"--store-queries"}, description = "Path to store the queries executed. Stores queries in the same order as the result sets are in the result files. Requires --results")
-    private String queryStorePath;
-
-    @Override
-    public void run()
-    {
-        try
-        {
-            List<File> resultPaths = null;
-            if (resultPath != null)
-            {
-                File basePath = new File(resultPath);
-                if (!basePath.exists() || !basePath.isDirectory())
-                {
-                    System.err.println("The results path (" + basePath + ") should be an existing directory");
-                    System.exit(1);
-                }
-                resultPaths = targetHosts.stream().map(target -> new File(basePath, target)).collect(Collectors.toList());
-                resultPaths.forEach(File::mkdir);
-            }
-            if (targetHosts.size() < 1)
-            {
-                System.err.println("You need to state at least one --target host to replay the query against");
-                System.exit(1);
-            }
-            replay(keyspace, arguments, targetHosts, resultPaths, queryStorePath, debug);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public static void replay(String keyspace, List<String> arguments, List<String> targetHosts, List<File> resultPaths, String queryStorePath, boolean debug)
-    {
-        int readAhead = 200; // how many fql queries should we read in to memory to be able to sort them?
-        List<ChronicleQueue> readQueues = null;
-        List<FQLQueryIterator> iterators = null;
-        List<Predicate<FQLQuery>> filters = new ArrayList<>();
-
-        if (keyspace != null)
-            filters.add(fqlQuery -> fqlQuery.keyspace == null || fqlQuery.keyspace.equals(keyspace));
-
-        try
-        {
-            readQueues = arguments.stream().map(s -> ChronicleQueueBuilder.single(s).readOnly(true).build()).collect(Collectors.toList());
-            iterators = readQueues.stream().map(ChronicleQueue::createTailer).map(tailer -> new FQLQueryIterator(tailer, readAhead)).collect(Collectors.toList());
-            try (MergeIterator<FQLQuery, List<FQLQuery>> iter = MergeIterator.get(iterators, FQLQuery::compareTo, new Reducer());
-                 QueryReplayer replayer = new QueryReplayer(iter, targetHosts, resultPaths, filters, System.out, queryStorePath, debug))
-            {
-                replayer.replay();
-            }
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-        finally
-        {
-            if (iterators != null)
-                iterators.forEach(AbstractIterator::close);
-            if (readQueues != null)
-                readQueues.forEach(Closeable::close);
-        }
-    }
-
-    @VisibleForTesting
-    public static class Reducer extends MergeIterator.Reducer<FQLQuery, List<FQLQuery>>
-    {
-        List<FQLQuery> queries = new ArrayList<>();
-        public void reduce(int idx, FQLQuery current)
-        {
-            queries.add(current);
-        }
-
-        protected List<FQLQuery> getReduced()
-        {
-            return queries;
-        }
-        protected void onKeyChange()
-        {
-            queries.clear();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/test/unit/org/apache/cassandra/tools/fqltool/FQLReplayTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/fqltool/FQLReplayTest.java b/test/unit/org/apache/cassandra/tools/fqltool/FQLReplayTest.java
deleted file mode 100644
index a662699..0000000
--- a/test/unit/org/apache/cassandra/tools/fqltool/FQLReplayTest.java
+++ /dev/null
@@ -1,760 +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.tools.fqltool;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Objects;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import org.junit.Test;
-
-import net.openhft.chronicle.queue.ChronicleQueue;
-import net.openhft.chronicle.queue.ChronicleQueueBuilder;
-import net.openhft.chronicle.queue.ExcerptAppender;
-import net.openhft.chronicle.queue.ExcerptTailer;
-import net.openhft.chronicle.wire.ValueIn;
-import org.apache.cassandra.audit.FullQueryLogger;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.statements.BatchStatement;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.tools.Util;
-import org.apache.cassandra.tools.fqltool.commands.Replay;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.MergeIterator;
-import org.apache.cassandra.utils.Pair;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-public class FQLReplayTest
-{
-    public FQLReplayTest()
-    {
-        Util.initDatabaseDescriptor();
-    }
-
-    @Test
-    public void testOrderedReplay() throws IOException
-    {
-        File f = generateQueries(100, true);
-        int queryCount = 0;
-        try (ChronicleQueue queue = ChronicleQueueBuilder.single(f).build();
-             FQLQueryIterator iter = new FQLQueryIterator(queue.createTailer(), 101))
-        {
-            long last = -1;
-            while (iter.hasNext())
-            {
-                FQLQuery q = iter.next();
-                assertTrue(q.queryStartTime >= last);
-                last = q.queryStartTime;
-                queryCount++;
-            }
-        }
-        assertEquals(100, queryCount);
-    }
-    @Test
-    public void testMergingIterator() throws IOException
-    {
-        File f = generateQueries(100, false);
-        File f2 = generateQueries(100, false);
-        int queryCount = 0;
-        try (ChronicleQueue queue = ChronicleQueueBuilder.single(f).build();
-             ChronicleQueue queue2 = ChronicleQueueBuilder.single(f2).build();
-             FQLQueryIterator iter = new FQLQueryIterator(queue.createTailer(), 101);
-             FQLQueryIterator iter2 = new FQLQueryIterator(queue2.createTailer(), 101);
-             MergeIterator<FQLQuery, List<FQLQuery>> merger = MergeIterator.get(Lists.newArrayList(iter, iter2), FQLQuery::compareTo, new Replay.Reducer()))
-        {
-            long last = -1;
-
-            while (merger.hasNext())
-            {
-                List<FQLQuery> qs = merger.next();
-                assertEquals(2, qs.size());
-                assertEquals(0, qs.get(0).compareTo(qs.get(1)));
-                assertTrue(qs.get(0).queryStartTime >= last);
-                last = qs.get(0).queryStartTime;
-                queryCount++;
-            }
-        }
-        assertEquals(100, queryCount);
-    }
-
-    @Test
-    public void testFQLQueryReader() throws IOException
-    {
-        FQLQueryReader reader = new FQLQueryReader();
-
-        try (ChronicleQueue queue = ChronicleQueueBuilder.single(generateQueries(1000, true)).build())
-        {
-            ExcerptTailer tailer = queue.createTailer();
-            int queryCount = 0;
-            while (tailer.readDocument(reader))
-            {
-                assertNotNull(reader.getQuery());
-                if (reader.getQuery() instanceof FQLQuery.Single)
-                {
-                    assertTrue(reader.getQuery().keyspace == null || reader.getQuery().keyspace.equals("querykeyspace"));
-                }
-                else
-                {
-                    assertEquals("someks", reader.getQuery().keyspace);
-                }
-                queryCount++;
-            }
-            assertEquals(1000, queryCount);
-        }
-    }
-
-    @Test
-    public void testStoringResults() throws Throwable
-    {
-        File tmpDir = Files.createTempDirectory("results").toFile();
-        File queryDir = Files.createTempDirectory("queries").toFile();
-
-        ResultHandler.ComparableResultSet res = createResultSet(10, 10, true);
-        ResultStore rs = new ResultStore(Collections.singletonList(tmpDir), queryDir);
-        try
-        {
-            FQLQuery query = new FQLQuery.Single("abc", 3, QueryOptions.DEFAULT, 12345, 11111, 22, "select * from abc", Collections.emptyList());
-            rs.storeColumnDefinitions(query, Collections.singletonList(res.getColumnDefinitions()));
-            Iterator<ResultHandler.ComparableRow> it = res.iterator();
-            while (it.hasNext())
-            {
-                List<ResultHandler.ComparableRow> row = Collections.singletonList(it.next());
-                rs.storeRows(row);
-            }
-            // this marks the end of the result set:
-            rs.storeRows(Collections.singletonList(null));
-        }
-        finally
-        {
-            rs.close();
-        }
-
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> resultSets = readResultFile(tmpDir, queryDir);
-        assertEquals(1, resultSets.size());
-        assertEquals(res, resultSets.get(0).right);
-
-    }
-
-    @Test
-    public void testCompareColumnDefinitions()
-    {
-        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
-        ResultComparator rc = new ResultComparator();
-
-        List<ResultHandler.ComparableColumnDefinitions> colDefs = new ArrayList<>(100);
-        List<String> targetHosts = new ArrayList<>(100);
-        for (int i = 0; i < 100; i++)
-        {
-            targetHosts.add("host"+i);
-            colDefs.add(res.getColumnDefinitions());
-        }
-        assertTrue(rc.compareColumnDefinitions(targetHosts, null, colDefs));
-        colDefs.set(50, createResultSet(9, 9, false).getColumnDefinitions());
-        assertFalse(rc.compareColumnDefinitions(targetHosts, null, colDefs));
-    }
-
-    @Test
-    public void testCompareEqualRows()
-    {
-        ResultComparator rc = new ResultComparator();
-
-        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
-        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
-        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2);
-        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
-
-        while (true)
-        {
-            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
-            assertTrue(rc.compareRows(Lists.newArrayList("eq1", "eq2"), null, rows));
-            if (rows.stream().allMatch(Objects::isNull))
-                break;
-        }
-    }
-
-    @Test
-    public void testCompareRowsDifferentCount()
-    {
-        ResultComparator rc = new ResultComparator();
-        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
-        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
-        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(10, 11, false));
-        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
-        boolean foundMismatch = false;
-        while (true)
-        {
-            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
-            if (rows.stream().allMatch(Objects::isNull))
-                break;
-            if (!rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows))
-            {
-                foundMismatch = true;
-            }
-        }
-        assertTrue(foundMismatch);
-    }
-
-    @Test
-    public void testCompareRowsDifferentContent()
-    {
-        ResultComparator rc = new ResultComparator();
-        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
-        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
-        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(10, 10, true));
-        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
-        while (true)
-        {
-            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
-            if (rows.stream().allMatch(Objects::isNull))
-                break;
-            assertFalse(rows.toString(), rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows));
-        }
-    }
-
-    @Test
-    public void testCompareRowsDifferentColumnCount()
-    {
-        ResultComparator rc = new ResultComparator();
-        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
-        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
-        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(11, 10, false));
-        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
-        while (true)
-        {
-            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
-            if (rows.stream().allMatch(Objects::isNull))
-                break;
-            assertFalse(rows.toString(), rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows));
-        }
-    }
-
-    @Test
-    public void testResultHandler() throws IOException
-    {
-        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
-        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
-        File queryDir = Files.createTempDirectory("queries").toFile();
-        List<File> resultPaths = new ArrayList<>();
-        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
-        ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir);
-        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
-        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
-        ResultHandler.ComparableResultSet res3 = createResultSet(10, 10, false);
-        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, res3);
-        rh.handleResults(new FQLQuery.Single("abcabc", 3, QueryOptions.DEFAULT, 1111, 2222, 3333, "select * from xyz", Collections.emptyList()), toCompare);
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(resultPaths.get(0), queryDir);
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results2 = readResultFile(resultPaths.get(1), queryDir);
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results3 = readResultFile(resultPaths.get(2), queryDir);
-        assertEquals(results1, results2);
-        assertEquals(results1, results3);
-        assertEquals(Iterables.getOnlyElement(results3).right, res);
-    }
-
-    @Test
-    public void testResultHandlerWithDifference() throws IOException
-    {
-        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
-        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
-        File queryDir = Files.createTempDirectory("queries").toFile();
-        List<File> resultPaths = new ArrayList<>();
-        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
-        ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir);
-        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
-        ResultHandler.ComparableResultSet res2 = createResultSet(10, 5, false);
-        ResultHandler.ComparableResultSet res3 = createResultSet(10, 10, false);
-        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, res3);
-        rh.handleResults(new FQLQuery.Single("aaa", 3, QueryOptions.DEFAULT, 123123, 11111, 22222, "select * from abcabc", Collections.emptyList()), toCompare);
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(resultPaths.get(0), queryDir);
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results2 = readResultFile(resultPaths.get(1), queryDir);
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results3 = readResultFile(resultPaths.get(2), queryDir);
-        assertEquals(results1, results3);
-        assertEquals(results2.get(0).right, res2);
-    }
-
-    @Test
-    public void testResultHandlerMultipleResultSets() throws IOException
-    {
-        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
-        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
-        File queryDir = Files.createTempDirectory("queries").toFile();
-        List<File> resultPaths = new ArrayList<>();
-        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
-        ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir);
-        List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets = new ArrayList<>();
-        Random random = new Random();
-        for (int i = 0; i < 10; i++)
-        {
-            List<ResultHandler.ComparableResultSet> results = new ArrayList<>();
-            List<ByteBuffer> values = Collections.singletonList(ByteBufferUtil.bytes(i * 50));
-            for (int jj = 0; jj < targetHosts.size(); jj++)
-            {
-                results.add(createResultSet(5, 1 + random.nextInt(10), true));
-            }
-            FQLQuery q = new FQLQuery.Single("abc"+i,
-                                             3,
-                                             QueryOptions.forInternalCalls(values),
-                                             i * 1000,
-                                             12345,
-                                             54321,
-                                             "select * from xyz where id = "+i,
-                                             values);
-            resultSets.add(Pair.create(q, results));
-        }
-        for (int i = 0; i < resultSets.size(); i++)
-            rh.handleResults(resultSets.get(i).left, resultSets.get(i).right);
-
-        for (int i = 0; i < targetHosts.size(); i++)
-            compareWithFile(resultPaths, queryDir, resultSets, i);
-    }
-
-    @Test
-    public void testResultHandlerFailedQuery() throws IOException
-    {
-        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc", "hostd");
-        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
-        File queryDir = Files.createTempDirectory("queries").toFile();
-        List<File> resultPaths = new ArrayList<>();
-        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
-        ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir);
-        List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets = new ArrayList<>();
-        Random random = new Random();
-        for (int i = 0; i < 10; i++)
-        {
-            List<ResultHandler.ComparableResultSet> results = new ArrayList<>();
-            List<ByteBuffer> values = Collections.singletonList(ByteBufferUtil.bytes(i * 50));
-            for (int jj = 0; jj < targetHosts.size(); jj++)
-            {
-                results.add(createResultSet(5, 1 + random.nextInt(10), true));
-            }
-            results.set(0, FakeResultSet.failed(new RuntimeException("testing abc")));
-            results.set(3, FakeResultSet.failed(new RuntimeException("testing abc")));
-            FQLQuery q = new FQLQuery.Single("abc"+i,
-                                             3,
-                                             QueryOptions.forInternalCalls(values),
-                                             i * 1000,
-                                             i * 12345,
-                                             i * 54321,
-                                             "select * from xyz where id = "+i,
-                                             values);
-            resultSets.add(Pair.create(q, results));
-        }
-        for (int i = 0; i < resultSets.size(); i++)
-            rh.handleResults(resultSets.get(i).left, resultSets.get(i).right);
-
-        for (int i = 0; i < targetHosts.size(); i++)
-            compareWithFile(resultPaths, queryDir, resultSets, i);
-    }
-
-
-    @Test
-    public void testCompare()
-    {
-        FQLQuery q1 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.emptyList());
-        FQLQuery q2 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222,"aaaa", Collections.emptyList());
-
-        assertEquals(0, q1.compareTo(q2));
-        assertEquals(0, q2.compareTo(q1));
-
-        FQLQuery q3 = new FQLQuery.Batch("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, com.datastax.driver.core.BatchStatement.Type.UNLOGGED, Collections.emptyList(), Collections.emptyList());
-        // single queries before batch queries
-        assertTrue(q1.compareTo(q3) < 0);
-        assertTrue(q3.compareTo(q1) > 0);
-
-        // check that smaller query time
-        FQLQuery q4 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 124, 111, 222, "aaaa", Collections.emptyList());
-        assertTrue(q1.compareTo(q4) < 0);
-        assertTrue(q4.compareTo(q1) > 0);
-
-        FQLQuery q5 = new FQLQuery.Batch("abc", 0, QueryOptions.DEFAULT, 124, 111, 222, com.datastax.driver.core.BatchStatement.Type.UNLOGGED, Collections.emptyList(), Collections.emptyList());
-        assertTrue(q1.compareTo(q5) < 0);
-        assertTrue(q5.compareTo(q1) > 0);
-
-        FQLQuery q6 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes(10)));
-        FQLQuery q7 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.emptyList());
-        assertTrue(q6.compareTo(q7) > 0);
-        assertTrue(q7.compareTo(q6) < 0);
-
-        FQLQuery q8 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes("a")));
-        FQLQuery q9 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes("b")));
-        assertTrue(q8.compareTo(q9) < 0);
-        assertTrue(q9.compareTo(q8) > 0);
-    }
-
-    private File generateQueries(int count, boolean random) throws IOException
-    {
-        Random r = new Random();
-        File dir = Files.createTempDirectory("chronicle").toFile();
-        try (ChronicleQueue readQueue = ChronicleQueueBuilder.single(dir).build())
-        {
-            ExcerptAppender appender = readQueue.acquireAppender();
-
-            for (int i = 0; i < count; i++)
-            {
-                long timestamp = random ? Math.abs(r.nextLong() % 10000) : i;
-                if (random ? r.nextBoolean() : i % 2 == 0)
-                {
-                    String query = "abcdefghijklm " + i;
-                    QueryState qs = r.nextBoolean() ? queryState() : queryState("querykeyspace");
-                    FullQueryLogger.Query  q = new FullQueryLogger.Query(query, QueryOptions.DEFAULT, qs, timestamp);
-                    appender.writeDocument(q);
-                    q.release();
-                }
-                else
-                {
-                    int batchSize = random ? r.nextInt(99) + 1 : i + 1;
-                    List<String> queries = new ArrayList<>(batchSize);
-                    List<List<ByteBuffer>> values = new ArrayList<>(batchSize);
-                    for (int jj = 0; jj < (random ? r.nextInt(batchSize) : 10); jj++)
-                    {
-                        queries.add("aaaaaa batch "+i+":"+jj);
-                        values.add(Collections.emptyList());
-                    }
-                    FullQueryLogger.Batch batch = new FullQueryLogger.Batch(BatchStatement.Type.UNLOGGED,
-                                                                            queries,
-                                                                            values,
-                                                                            QueryOptions.DEFAULT,
-                                                                            queryState("someks"),
-                                                                            timestamp);
-                    appender.writeDocument(batch);
-                    batch.release();
-                }
-            }
-        }
-        return dir;
-    }
-
-    private QueryState queryState()
-    {
-        return QueryState.forInternalCalls();
-    }
-
-    private QueryState queryState(String keyspace)
-    {
-        ClientState clientState = ClientState.forInternalCalls(keyspace);
-        return new QueryState(clientState);
-    }
-
-    private static ResultHandler.ComparableResultSet createResultSet(int columnCount, int rowCount, boolean random)
-    {
-        List<Pair<String, String>> columnDefs = new ArrayList<>(columnCount);
-        Random r = new Random();
-        for (int i = 0; i < columnCount; i++)
-        {
-            columnDefs.add(Pair.create("a" + i, "int"));
-        }
-        List<List<String>> rows = new ArrayList<>();
-        for (int i = 0; i < rowCount; i++)
-        {
-            List<String> row = new ArrayList<>(columnCount);
-            for (int jj = 0; jj < columnCount; jj++)
-                row.add(i + " col " + jj + (random ? r.nextInt() : ""));
-            rows.add(row);
-        }
-        return new FakeResultSet(columnDefs, rows);
-    }
-
-    private static void compareWithFile(List<File> dirs, File resultDir, List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets, int idx)
-    {
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(dirs.get(idx), resultDir);
-        for (int i = 0; i < results1.size(); i++)
-        {
-            assertEquals(results1.get(i).left, resultSets.get(i).left);
-            assertEquals(results1.get(i).right, resultSets.get(i).right.get(idx));
-        }
-    }
-
-    private static List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> readResultFile(File dir, File queryDir)
-    {
-        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> resultSets = new ArrayList<>();
-        try (ChronicleQueue q = ChronicleQueueBuilder.single(dir).build();
-             ChronicleQueue queryQ = ChronicleQueueBuilder.single(queryDir).build())
-        {
-            ExcerptTailer tailer = q.createTailer();
-            ExcerptTailer queryTailer = queryQ.createTailer();
-            List<Pair<String, String>> columnDefinitions = new ArrayList<>();
-            List<List<String>> rowColumns = new ArrayList<>();
-            AtomicBoolean allRowsRead = new AtomicBoolean(false);
-            AtomicBoolean failedQuery = new AtomicBoolean(false);
-            while (tailer.readDocument(wire -> {
-                String type = wire.read("type").text();
-                if (type.equals("column_definitions"))
-                {
-                    int columnCount = wire.read("column_count").int32();
-                    for (int i = 0; i < columnCount; i++)
-                    {
-                        ValueIn vi = wire.read("column_definition");
-                        String name = vi.text();
-                        String dataType = vi.text();
-                        columnDefinitions.add(Pair.create(name, dataType));
-                    }
-                }
-                else if (type.equals("row"))
-                {
-                    int rowColumnCount = wire.read("row_column_count").int32();
-                    List<String> r = new ArrayList<>(rowColumnCount);
-                    for (int i = 0; i < rowColumnCount; i++)
-                    {
-                        byte[] b = wire.read("column").bytes();
-                        r.add(new String(b));
-                    }
-                    rowColumns.add(r);
-                }
-                else if (type.equals("end_resultset"))
-                {
-                    allRowsRead.set(true);
-                }
-                else if (type.equals("query_failed"))
-                {
-                    failedQuery.set(true);
-                }
-            }))
-            {
-                if (allRowsRead.get())
-                {
-                    FQLQueryReader reader = new FQLQueryReader();
-                    queryTailer.readDocument(reader);
-                    resultSets.add(Pair.create(reader.getQuery(), failedQuery.get() ? FakeResultSet.failed(new RuntimeException("failure"))
-                                                                                    : new FakeResultSet(ImmutableList.copyOf(columnDefinitions), ImmutableList.copyOf(rowColumns))));
-                    allRowsRead.set(false);
-                    failedQuery.set(false);
-                    columnDefinitions.clear();
-                    rowColumns.clear();
-                }
-            }
-        }
-        return resultSets;
-    }
-
-    private static class FakeResultSet implements ResultHandler.ComparableResultSet
-    {
-        private final List<Pair<String, String>> cdStrings;
-        private final List<List<String>> rows;
-        private final Throwable ex;
-
-        public FakeResultSet(List<Pair<String, String>> cdStrings, List<List<String>> rows)
-        {
-            this(cdStrings, rows, null);
-        }
-
-        public FakeResultSet(List<Pair<String, String>> cdStrings, List<List<String>> rows, Throwable ex)
-        {
-            this.cdStrings = cdStrings;
-            this.rows = rows;
-            this.ex = ex;
-        }
-
-        public static FakeResultSet failed(Throwable ex)
-        {
-            return new FakeResultSet(null, null, ex);
-        }
-
-        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
-        {
-            return new FakeComparableColumnDefinitions(cdStrings, wasFailed());
-        }
-
-        public boolean wasFailed()
-        {
-            return getFailureException() != null;
-        }
-
-        public Throwable getFailureException()
-        {
-            return ex;
-        }
-
-        public Iterator<ResultHandler.ComparableRow> iterator()
-        {
-            if (wasFailed())
-                return Collections.emptyListIterator();
-            return new AbstractIterator<ResultHandler.ComparableRow>()
-            {
-                Iterator<List<String>> iter = rows.iterator();
-                protected ResultHandler.ComparableRow computeNext()
-                {
-                    if (iter.hasNext())
-                        return new FakeComparableRow(iter.next(), cdStrings);
-                    return endOfData();
-                }
-            };
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (!(o instanceof FakeResultSet)) return false;
-            FakeResultSet that = (FakeResultSet) o;
-            if (wasFailed() && that.wasFailed())
-                return true;
-            return Objects.equals(cdStrings, that.cdStrings) &&
-                   Objects.equals(rows, that.rows);
-        }
-
-        public int hashCode()
-        {
-            return Objects.hash(cdStrings, rows);
-        }
-
-        public String toString()
-        {
-            return "FakeResultSet{" +
-                   "cdStrings=" + cdStrings +
-                   ", rows=" + rows +
-                   '}';
-        }
-    }
-
-    private static class FakeComparableRow implements ResultHandler.ComparableRow
-    {
-        private final List<String> row;
-        private final List<Pair<String, String>> cds;
-
-        public FakeComparableRow(List<String> row, List<Pair<String,String>> cds)
-        {
-            this.row = row;
-            this.cds = cds;
-        }
-
-        public ByteBuffer getBytesUnsafe(int i)
-        {
-            return ByteBufferUtil.bytes(row.get(i));
-        }
-
-        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
-        {
-            return new FakeComparableColumnDefinitions(cds, false);
-        }
-
-        public boolean equals(Object other)
-        {
-            if (!(other instanceof FakeComparableRow))
-                return false;
-            return row.equals(((FakeComparableRow)other).row);
-        }
-
-        public String toString()
-        {
-            return row.toString();
-        }
-    }
-
-    private static class FakeComparableColumnDefinitions implements ResultHandler.ComparableColumnDefinitions
-    {
-        private final List<ResultHandler.ComparableDefinition> defs;
-        private final boolean failed;
-        public FakeComparableColumnDefinitions(List<Pair<String, String>> cds, boolean failed)
-        {
-            defs = cds != null ? cds.stream().map(FakeComparableDefinition::new).collect(Collectors.toList()) : null;
-            this.failed = failed;
-        }
-
-        public List<ResultHandler.ComparableDefinition> asList()
-        {
-            if (wasFailed())
-                return Collections.emptyList();
-            return defs;
-        }
-
-        public boolean wasFailed()
-        {
-            return failed;
-        }
-
-        public int size()
-        {
-            return defs.size();
-        }
-
-        public Iterator<ResultHandler.ComparableDefinition> iterator()
-        {
-            if (wasFailed())
-                return Collections.emptyListIterator();
-            return new AbstractIterator<ResultHandler.ComparableDefinition>()
-            {
-                Iterator<ResultHandler.ComparableDefinition> iter = defs.iterator();
-                protected ResultHandler.ComparableDefinition computeNext()
-                {
-                    if (iter.hasNext())
-                        return iter.next();
-                    return endOfData();
-                }
-            };
-        }
-        public boolean equals(Object other)
-        {
-            if (!(other instanceof FakeComparableColumnDefinitions))
-                return false;
-            return defs.equals(((FakeComparableColumnDefinitions)other).defs);
-        }
-
-        public String toString()
-        {
-            return defs.toString();
-        }
-    }
-
-    private static class FakeComparableDefinition implements ResultHandler.ComparableDefinition
-    {
-        private final Pair<String, String> p;
-
-        public FakeComparableDefinition(Pair<String, String> p)
-        {
-            this.p = p;
-        }
-        public String getType()
-        {
-            return p.right;
-        }
-
-        public String getName()
-        {
-            return p.left;
-        }
-
-        public boolean equals(Object other)
-        {
-            if (!(other instanceof FakeComparableDefinition))
-                return false;
-            return p.equals(((FakeComparableDefinition)other).p);
-        }
-
-        public String toString()
-        {
-            return getName() + ':' + getType();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/bin/cassandra.in.bat
----------------------------------------------------------------------
diff --git a/tools/bin/cassandra.in.bat b/tools/bin/cassandra.in.bat
index 8804921..4d395e8 100644
--- a/tools/bin/cassandra.in.bat
+++ b/tools/bin/cassandra.in.bat
@@ -39,7 +39,7 @@ goto :eof
 :okClasspath
 
 REM Include the build\classes\main directory so it works in development
-set CASSANDRA_CLASSPATH=%CLASSPATH%;%CASSANDRA_CONF%;"%CASSANDRA_HOME%\build\classes\main";"%CASSANDRA_HOME%\build\classes\stress"
+set CASSANDRA_CLASSPATH=%CLASSPATH%;%CASSANDRA_CONF%;"%CASSANDRA_HOME%\build\classes\main";"%CASSANDRA_HOME%\build\classes\stress";"%CASSANDRA_HOME%\build\classes\fqltool"
 
 REM Add the default storage location.  Can be overridden in conf\cassandra.yaml
 set CASSANDRA_PARAMS=%CASSANDRA_PARAMS% "-Dcassandra.storagedir=%CASSANDRA_HOME%\data"

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/bin/cassandra.in.sh
----------------------------------------------------------------------
diff --git a/tools/bin/cassandra.in.sh b/tools/bin/cassandra.in.sh
index 869eb22..bf1ecc4 100644
--- a/tools/bin/cassandra.in.sh
+++ b/tools/bin/cassandra.in.sh
@@ -32,7 +32,7 @@ CLASSPATH="$CASSANDRA_CONF"
 if [ -d $CASSANDRA_HOME/build ] ; then
     #cassandra_bin="$CASSANDRA_HOME/build/classes/main"
     cassandra_bin=`ls -1 $CASSANDRA_HOME/build/apache-cassandra*.jar`
-    cassandra_bin="$cassandra_bin:$CASSANDRA_HOME/build/classes/stress"
+    cassandra_bin="$cassandra_bin:$CASSANDRA_HOME/build/classes/stress:$CASSANDRA_HOME/build/classes/fqltool"
     CLASSPATH="$CLASSPATH:$cassandra_bin"
 fi
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/bin/fqltool
----------------------------------------------------------------------
diff --git a/tools/bin/fqltool b/tools/bin/fqltool
new file mode 100755
index 0000000..a34128e
--- /dev/null
+++ b/tools/bin/fqltool
@@ -0,0 +1,76 @@
+#!/bin/sh
+
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+if [ "x$CASSANDRA_INCLUDE" = "x" ]; then
+    # Locations (in order) to use when searching for an include file.
+    for include in "`dirname "$0"`/cassandra.in.sh" \
+                   "$HOME/.cassandra.in.sh" \
+                   /usr/share/cassandra/cassandra.in.sh \
+                   /usr/local/share/cassandra/cassandra.in.sh \
+                   /opt/cassandra/cassandra.in.sh; do
+        if [ -r "$include" ]; then
+            . "$include"
+            break
+        fi
+    done
+elif [ -r "$CASSANDRA_INCLUDE" ]; then
+    . "$CASSANDRA_INCLUDE"
+fi
+
+if [ -z "$CASSANDRA_CONF" -o -z "$CLASSPATH" ]; then
+    echo "You must set the CASSANDRA_CONF and CLASSPATH vars" >&2
+    exit 1
+fi
+
+# Run cassandra-env.sh to pick up JMX_PORT
+if [ -f "$CASSANDRA_CONF/cassandra-env.sh" ]; then
+    JVM_OPTS_SAVE=$JVM_OPTS
+    MAX_HEAP_SIZE_SAVE=$MAX_HEAP_SIZE
+    . "$CASSANDRA_CONF/cassandra-env.sh"
+    MAX_HEAP_SIZE=$MAX_HEAP_SIZE_SAVE
+    JVM_OPTS=$JVM_OPTS_SAVE
+fi
+
+# JMX Port passed via cmd line args (-p 9999 / --port 9999 / --port=9999)
+# should override the value from cassandra-env.sh
+ARGS=""
+JVM_ARGS=""
+while true
+do
+  if [ ! $1 ]; then break; fi
+  case $1 in
+    -D*)
+      JVM_ARGS="$JVM_ARGS $1"
+      ;;
+    *)
+      ARGS="$ARGS $1"
+      ;;
+  esac
+  shift
+done
+
+if [ "x$MAX_HEAP_SIZE" = "x" ]; then
+    MAX_HEAP_SIZE="512m"
+fi
+
+"$JAVA" $JAVA_AGENT -ea -da:net.openhft... -cp "$CLASSPATH" $JVM_OPTS -Xmx$MAX_HEAP_SIZE \
+        -Dlog4j.configurationFile=log4j2-tools.xml \
+        $JVM_ARGS \
+        org.apache.cassandra.fqltool.FullQueryLogTool $ARGS
+
+# vi:ai sw=4 ts=4 tw=0 et

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/bin/fqltool.bat
----------------------------------------------------------------------
diff --git a/tools/bin/fqltool.bat b/tools/bin/fqltool.bat
new file mode 100644
index 0000000..acb6d1c
--- /dev/null
+++ b/tools/bin/fqltool.bat
@@ -0,0 +1,36 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one or more
+@REM contributor license agreements. See the NOTICE file distributed with
+@REM this work for additional information regarding copyright ownership.
+@REM The ASF licenses this file to You under the Apache License, Version 2.0
+@REM (the "License"); you may not use this file except in compliance with
+@REM the License. You may obtain a copy of the License at
+@REM
+@REM http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing, software
+@REM distributed under the License is distributed on an "AS IS" BASIS,
+@REM WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@REM See the License for the specific language governing permissions and
+@REM limitations under the License.
+
+@echo off
+if "%OS%" == "Windows_NT" setlocal
+
+pushd "%~dp0"
+call cassandra.in.bat
+
+if NOT DEFINED JAVA_HOME goto :err
+
+set CASSANDRA_PARAMS=%CASSANDRA_PARAMS% -Dcassandra.logdir="%CASSANDRA_HOME%\logs"
+
+"%JAVA_HOME%\bin\java" -cp %CASSANDRA_CLASSPATH% %CASSANDRA_PARAMS% -Dlog4j.configurationFile=log4j2-tools.xml org.apache.cassandra.fqltool.FullQueryLogTool %*
+goto finally
+
+:err
+echo The JAVA_HOME environment variable must be set to run this program!
+pause
+
+:finally
+ENDLOCAL & set RC=%ERRORLEVEL%
+exit /B %RC%

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/DriverResultSet.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/DriverResultSet.java b/tools/fqltool/src/org/apache/cassandra/fqltool/DriverResultSet.java
new file mode 100644
index 0000000..ccff370
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/DriverResultSet.java
@@ -0,0 +1,248 @@
+/*
+ * 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.fqltool;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.AbstractIterator;
+
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+/**
+ * Wraps a result set from the driver so that we can reuse the compare code when reading
+ * up a result set produced by ResultStore.
+ */
+public class DriverResultSet implements ResultHandler.ComparableResultSet
+{
+    private final ResultSet resultSet;
+    private final Throwable failureException;
+
+    public DriverResultSet(ResultSet resultSet)
+    {
+        this(resultSet, null);
+    }
+
+    private DriverResultSet(ResultSet res, Throwable failureException)
+    {
+        resultSet = res;
+        this.failureException = failureException;
+    }
+
+    public static DriverResultSet failed(Throwable ex)
+    {
+        return new DriverResultSet(null, ex);
+    }
+
+    public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+    {
+        if (wasFailed())
+            return new DriverColumnDefinitions(null, true, failureException);
+
+        return new DriverColumnDefinitions(resultSet.getColumnDefinitions());
+    }
+
+    public boolean wasFailed()
+    {
+        return failureException != null;
+    }
+
+    public Throwable getFailureException()
+    {
+        return failureException;
+    }
+
+    public Iterator<ResultHandler.ComparableRow> iterator()
+    {
+        if (wasFailed())
+            return Collections.emptyListIterator();
+        return new AbstractIterator<ResultHandler.ComparableRow>()
+        {
+            Iterator<Row> iter = resultSet.iterator();
+            protected ResultHandler.ComparableRow computeNext()
+            {
+                if (iter.hasNext())
+                    return new DriverRow(iter.next());
+                return endOfData();
+            }
+        };
+    }
+
+    public static class DriverRow implements ResultHandler.ComparableRow
+    {
+        private final Row row;
+
+        public DriverRow(Row row)
+        {
+            this.row = row;
+        }
+
+        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+        {
+            return new DriverColumnDefinitions(row.getColumnDefinitions());
+        }
+
+        public ByteBuffer getBytesUnsafe(int i)
+        {
+            return row.getBytesUnsafe(i);
+        }
+
+        @Override
+        public boolean equals(Object oo)
+        {
+            if (!(oo instanceof ResultHandler.ComparableRow))
+                return false;
+
+            ResultHandler.ComparableRow o = (ResultHandler.ComparableRow)oo;
+            if (getColumnDefinitions().size() != o.getColumnDefinitions().size())
+                return false;
+
+            for (int j = 0; j < getColumnDefinitions().size(); j++)
+            {
+                ByteBuffer b1 = getBytesUnsafe(j);
+                ByteBuffer b2 = o.getBytesUnsafe(j);
+
+                if (b1 != null && b2 != null && !b1.equals(b2))
+                {
+                    return false;
+                }
+                if (b1 == null && b2 != null || b2 == null && b1 != null)
+                {
+                    return false;
+                }
+            }
+            return true;
+        }
+
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder();
+            List<ResultHandler.ComparableDefinition> colDefs = getColumnDefinitions().asList();
+            for (int i = 0; i < getColumnDefinitions().size(); i++)
+            {
+                ByteBuffer bb = getBytesUnsafe(i);
+                String row = bb != null ? ByteBufferUtil.bytesToHex(bb) : "NULL";
+                sb.append(colDefs.get(i)).append(':').append(row).append(",");
+            }
+            return sb.toString();
+        }
+    }
+
+    public static class DriverColumnDefinitions implements ResultHandler.ComparableColumnDefinitions
+    {
+        private final ColumnDefinitions columnDefinitions;
+        private final boolean failed;
+        private final Throwable failureException;
+
+        public DriverColumnDefinitions(ColumnDefinitions columnDefinitions)
+        {
+            this(columnDefinitions, false, null);
+        }
+
+        private DriverColumnDefinitions(ColumnDefinitions columnDefinitions, boolean failed, Throwable failureException)
+        {
+            this.columnDefinitions = columnDefinitions;
+            this.failed = failed;
+            this.failureException = failureException;
+        }
+
+        public List<ResultHandler.ComparableDefinition> asList()
+        {
+            if (wasFailed())
+                return Collections.emptyList();
+            return columnDefinitions.asList().stream().map(DriverDefinition::new).collect(Collectors.toList());
+        }
+
+        public boolean wasFailed()
+        {
+            return failed;
+        }
+
+        public Throwable getFailureException()
+        {
+            return failureException;
+        }
+
+        public int size()
+        {
+            return columnDefinitions.size();
+        }
+
+        public Iterator<ResultHandler.ComparableDefinition> iterator()
+        {
+            return asList().iterator();
+        }
+
+        public boolean equals(Object oo)
+        {
+            if (!(oo instanceof ResultHandler.ComparableColumnDefinitions))
+                return false;
+
+            ResultHandler.ComparableColumnDefinitions o = (ResultHandler.ComparableColumnDefinitions)oo;
+            if (wasFailed() && o.wasFailed())
+                return true;
+
+            if (size() != o.size())
+                return false;
+
+            return asList().equals(o.asList());
+        }
+    }
+
+    public static class DriverDefinition implements ResultHandler.ComparableDefinition
+    {
+        private final ColumnDefinitions.Definition def;
+
+        public DriverDefinition(ColumnDefinitions.Definition def)
+        {
+            this.def = def;
+        }
+
+        public String getType()
+        {
+            return def.getType().toString();
+        }
+
+        public String getName()
+        {
+            return def.getName();
+        }
+
+        public boolean equals(Object oo)
+        {
+            if (!(oo instanceof ResultHandler.ComparableDefinition))
+                return false;
+
+            return def.equals(((DriverDefinition)oo).def);
+        }
+
+        public String toString()
+        {
+            return getName() + ':' + getType();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQuery.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQuery.java b/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQuery.java
new file mode 100644
index 0000000..2862e0f
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQuery.java
@@ -0,0 +1,265 @@
+/*
+ * 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.fqltool;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import com.google.common.primitives.Longs;
+
+import com.datastax.driver.core.BatchStatement;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.audit.FullQueryLogger;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.binlog.BinLog;
+
+public abstract class FQLQuery implements Comparable<FQLQuery>
+{
+    public final long queryStartTime;
+    public final QueryOptions queryOptions;
+    public final int protocolVersion;
+    public final QueryState queryState;
+
+    public FQLQuery(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds)
+    {
+        this.queryStartTime = queryStartTime;
+        this.queryOptions = queryOptions;
+        this.protocolVersion = protocolVersion;
+        this.queryState = queryState(keyspace, generatedTimestamp, generatedNowInSeconds);
+    }
+
+    public abstract Statement toStatement();
+
+    /**
+     * used when storing the queries executed
+     */
+    public abstract BinLog.ReleaseableWriteMarshallable toMarshallable();
+
+    public String keyspace()
+    {
+        return queryState.getClientState().getRawKeyspace();
+    }
+
+    private QueryState queryState(String keyspace, long generatedTimestamp, int generatedNowInSeconds)
+    {
+        ClientState clientState = keyspace != null ? ClientState.forInternalCalls(keyspace) : ClientState.forInternalCalls();
+        return new QueryState(clientState, generatedTimestamp, generatedNowInSeconds);
+    }
+
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (!(o instanceof FQLQuery)) return false;
+        FQLQuery fqlQuery = (FQLQuery) o;
+        return queryStartTime == fqlQuery.queryStartTime &&
+               protocolVersion == fqlQuery.protocolVersion &&
+               queryState.getTimestamp() == fqlQuery.queryState.getTimestamp() &&
+               Objects.equals(queryState.getClientState().getRawKeyspace(), fqlQuery.queryState.getClientState().getRawKeyspace()) &&
+               Objects.equals(queryOptions.getValues(), fqlQuery.queryOptions.getValues());
+    }
+
+    public int hashCode()
+    {
+        return Objects.hash(queryStartTime, queryOptions, protocolVersion, queryState.getClientState().getRawKeyspace());
+    }
+
+    public int compareTo(FQLQuery other)
+    {
+        return Longs.compare(queryStartTime, other.queryStartTime);
+    }
+
+    public String toString()
+    {
+        return "FQLQuery{" +
+               "queryStartTime=" + queryStartTime +
+               ", protocolVersion=" + protocolVersion +
+               ", queryState='" + queryState + '\'' +
+               '}';
+    }
+
+    public static class Single extends FQLQuery
+    {
+        public final String query;
+        public final List<ByteBuffer> values;
+
+        public Single(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds, String queryString, List<ByteBuffer> values)
+        {
+            super(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds);
+            this.query = queryString;
+            this.values = values;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("%s%nQuery = %s, Values = %s",
+                                 super.toString(),
+                                 query,
+                                 values.stream().map(ByteBufferUtil::bytesToHex).collect(Collectors.joining(",")));
+        }
+
+        public Statement toStatement()
+        {
+            SimpleStatement ss = new SimpleStatement(query, values.toArray());
+            ss.setConsistencyLevel(ConsistencyLevel.valueOf(queryOptions.getConsistency().name()));
+            ss.setDefaultTimestamp(queryOptions.getTimestamp(queryState));
+            return ss;
+        }
+
+        public BinLog.ReleaseableWriteMarshallable toMarshallable()
+        {
+
+            return new FullQueryLogger.Query(query, queryOptions, queryState, queryStartTime);
+        }
+
+        public int compareTo(FQLQuery other)
+        {
+            int cmp = super.compareTo(other);
+
+            if (cmp == 0)
+            {
+                if (other instanceof Batch)
+                    return -1;
+
+                Single singleQuery = (Single) other;
+
+                cmp = query.compareTo(singleQuery.query);
+                if (cmp == 0)
+                {
+                    if (values.size() != singleQuery.values.size())
+                        return values.size() - singleQuery.values.size();
+                    for (int i = 0; i < values.size(); i++)
+                    {
+                        cmp = values.get(i).compareTo(singleQuery.values.get(i));
+                        if (cmp != 0)
+                            return cmp;
+                    }
+                }
+            }
+            return cmp;
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (!(o instanceof Single)) return false;
+            if (!super.equals(o)) return false;
+            Single single = (Single) o;
+            return Objects.equals(query, single.query) &&
+                   Objects.equals(values, single.values);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(super.hashCode(), query, values);
+        }
+    }
+
+    public static class Batch extends FQLQuery
+    {
+        public final BatchStatement.Type batchType;
+        public final List<Single> queries;
+
+        public Batch(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds, BatchStatement.Type batchType, List<String> queries, List<List<ByteBuffer>> values)
+        {
+            super(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds);
+            this.batchType = batchType;
+            this.queries = new ArrayList<>(queries.size());
+            for (int i = 0; i < queries.size(); i++)
+                this.queries.add(new Single(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds, queries.get(i), values.get(i)));
+        }
+
+        public Statement toStatement()
+        {
+            BatchStatement bs = new BatchStatement(batchType);
+            for (Single query : queries)
+                bs.add(query.toStatement());
+            bs.setConsistencyLevel(ConsistencyLevel.valueOf(queryOptions.getConsistency().name()));
+            bs.setDefaultTimestamp(queryOptions.getTimestamp(queryState));
+            return bs;
+        }
+
+        public int compareTo(FQLQuery other)
+        {
+            int cmp = super.compareTo(other);
+
+            if (cmp == 0)
+            {
+                if (other instanceof Single)
+                    return 1;
+
+                Batch otherBatch = (Batch) other;
+                if (queries.size() != otherBatch.queries.size())
+                    return queries.size() - otherBatch.queries.size();
+                for (int i = 0; i < queries.size(); i++)
+                {
+                    cmp = queries.get(i).compareTo(otherBatch.queries.get(i));
+                    if (cmp != 0)
+                        return cmp;
+                }
+            }
+            return cmp;
+        }
+
+        public BinLog.ReleaseableWriteMarshallable toMarshallable()
+        {
+            List<String> queryStrings = new ArrayList<>();
+            List<List<ByteBuffer>> values = new ArrayList<>();
+            for (Single q : queries)
+            {
+                queryStrings.add(q.query);
+                values.add(q.values);
+            }
+            return new FullQueryLogger.Batch(org.apache.cassandra.cql3.statements.BatchStatement.Type.valueOf(batchType.name()), queryStrings, values, queryOptions, queryState, queryStartTime);
+        }
+
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder(super.toString()).append("\nbatch: ").append(batchType).append('\n');
+            for (Single q : queries)
+                sb.append(q.toString()).append('\n');
+            sb.append("end batch");
+            return sb.toString();
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (!(o instanceof Batch)) return false;
+            if (!super.equals(o)) return false;
+            Batch batch = (Batch) o;
+            return batchType == batch.batchType &&
+                   Objects.equals(queries, batch.queries);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(super.hashCode(), batchType, queries);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryIterator.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryIterator.java b/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryIterator.java
new file mode 100644
index 0000000..ccbb200
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryIterator.java
@@ -0,0 +1,72 @@
+/*
+ * 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.fqltool;
+
+import java.util.PriorityQueue;
+
+import net.openhft.chronicle.queue.ExcerptTailer;
+import org.apache.cassandra.utils.AbstractIterator;
+
+public class FQLQueryIterator extends AbstractIterator<FQLQuery>
+{
+    // use a priority queue to be able to sort the head of the query logs in memory
+    private final PriorityQueue<FQLQuery> pq;
+    private final ExcerptTailer tailer;
+    private final FQLQueryReader reader;
+
+    /**
+     * Create an iterator over the FQLQueries in tailer
+     *
+     * Reads up to readAhead queries in to memory to be able to sort them (the files are mostly sorted already)
+     */
+    public FQLQueryIterator(ExcerptTailer tailer, int readAhead)
+    {
+        assert readAhead > 0 : "readAhead needs to be > 0";
+        reader = new FQLQueryReader();
+        this.tailer = tailer;
+        pq = new PriorityQueue<>(readAhead);
+        for (int i = 0; i < readAhead; i++)
+        {
+            FQLQuery next = readNext();
+            if (next != null)
+                pq.add(next);
+            else
+                break;
+        }
+    }
+
+    protected FQLQuery computeNext()
+    {
+        FQLQuery q = pq.poll();
+        if (q == null)
+            return endOfData();
+        FQLQuery next = readNext();
+        if (next != null)
+            pq.add(next);
+        return q;
+    }
+
+    private FQLQuery readNext()
+    {
+        if (tailer.readDocument(reader))
+            return reader.getQuery();
+        return null;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryReader.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryReader.java b/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryReader.java
new file mode 100644
index 0000000..fd5073c
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryReader.java
@@ -0,0 +1,116 @@
+/*
+ * 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.fqltool;
+
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.datastax.driver.core.BatchStatement;
+import io.netty.buffer.Unpooled;
+import net.openhft.chronicle.core.io.IORuntimeException;
+import net.openhft.chronicle.wire.ReadMarshallable;
+import net.openhft.chronicle.wire.ValueIn;
+import net.openhft.chronicle.wire.WireIn;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static org.apache.cassandra.audit.FullQueryLogger.GENERATED_NOW_IN_SECONDS;
+import static org.apache.cassandra.audit.FullQueryLogger.GENERATED_TIMESTAMP;
+import static org.apache.cassandra.audit.FullQueryLogger.KEYSPACE;
+import static org.apache.cassandra.audit.FullQueryLogger.PROTOCOL_VERSION;
+import static org.apache.cassandra.audit.FullQueryLogger.QUERY_OPTIONS;
+import static org.apache.cassandra.audit.FullQueryLogger.QUERY_START_TIME;
+import static org.apache.cassandra.audit.FullQueryLogger.TYPE;
+import static org.apache.cassandra.audit.FullQueryLogger.VERSION;
+import static org.apache.cassandra.audit.FullQueryLogger.BATCH;
+import static org.apache.cassandra.audit.FullQueryLogger.BATCH_TYPE;
+import static org.apache.cassandra.audit.FullQueryLogger.QUERIES;
+import static org.apache.cassandra.audit.FullQueryLogger.QUERY;
+import static org.apache.cassandra.audit.FullQueryLogger.SINGLE_QUERY;
+import static org.apache.cassandra.audit.FullQueryLogger.VALUES;
+
+public class FQLQueryReader implements ReadMarshallable
+{
+    private FQLQuery query;
+
+    public void readMarshallable(WireIn wireIn) throws IORuntimeException
+    {
+        int currentVersion = wireIn.read(VERSION).int16();
+        String type = wireIn.read(TYPE).text();
+        long queryStartTime = wireIn.read(QUERY_START_TIME).int64();
+        int protocolVersion = wireIn.read(PROTOCOL_VERSION).int32();
+        QueryOptions queryOptions = QueryOptions.codec.decode(Unpooled.wrappedBuffer(wireIn.read(QUERY_OPTIONS).bytes()), ProtocolVersion.decode(protocolVersion));
+        long generatedTimestamp = wireIn.read(GENERATED_TIMESTAMP).int64();
+        int generatedNowInSeconds = wireIn.read(GENERATED_NOW_IN_SECONDS).int32();
+        String keyspace = wireIn.read(KEYSPACE).text();
+
+        switch (type)
+        {
+            case SINGLE_QUERY:
+                String queryString = wireIn.read(QUERY).text();
+                query = new FQLQuery.Single(keyspace,
+                                            protocolVersion,
+                                            queryOptions,
+                                            queryStartTime,
+                                            generatedTimestamp,
+                                            generatedNowInSeconds,
+                                            queryString,
+                                            queryOptions.getValues());
+                break;
+            case BATCH:
+                BatchStatement.Type batchType = BatchStatement.Type.valueOf(wireIn.read(BATCH_TYPE).text());
+                ValueIn in = wireIn.read(QUERIES);
+                int queryCount = in.int32();
+
+                List<String> queries = new ArrayList<>(queryCount);
+                for (int i = 0; i < queryCount; i++)
+                    queries.add(in.text());
+                in = wireIn.read(VALUES);
+                int valueCount = in.int32();
+                List<List<ByteBuffer>> values = new ArrayList<>(valueCount);
+                for (int ii = 0; ii < valueCount; ii++)
+                {
+                    List<ByteBuffer> subValues = new ArrayList<>();
+                    values.add(subValues);
+                    int numSubValues = in.int32();
+                    for (int zz = 0; zz < numSubValues; zz++)
+                        subValues.add(ByteBuffer.wrap(in.bytes()));
+                }
+                query = new FQLQuery.Batch(keyspace,
+                                           protocolVersion,
+                                           queryOptions,
+                                           queryStartTime,
+                                           generatedTimestamp,
+                                           generatedNowInSeconds,
+                                           batchType,
+                                           queries,
+                                           values);
+                break;
+            default:
+                throw new RuntimeException("Unknown type: " + type);
+        }
+    }
+
+    public FQLQuery getQuery()
+    {
+        return query;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/FullQueryLogTool.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/FullQueryLogTool.java b/tools/fqltool/src/org/apache/cassandra/fqltool/FullQueryLogTool.java
new file mode 100644
index 0000000..97e7487
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/FullQueryLogTool.java
@@ -0,0 +1,99 @@
+/*
+ * 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.fqltool;
+
+import java.util.List;
+
+import com.google.common.base.Throwables;
+
+import io.airlift.airline.Cli;
+import io.airlift.airline.Help;
+import io.airlift.airline.ParseArgumentsMissingException;
+import io.airlift.airline.ParseArgumentsUnexpectedException;
+import io.airlift.airline.ParseCommandMissingException;
+import io.airlift.airline.ParseCommandUnrecognizedException;
+import io.airlift.airline.ParseOptionConversionException;
+import io.airlift.airline.ParseOptionMissingException;
+import io.airlift.airline.ParseOptionMissingValueException;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.fqltool.commands.Compare;
+import org.apache.cassandra.fqltool.commands.Dump;
+import org.apache.cassandra.fqltool.commands.Replay;
+
+import static com.google.common.base.Throwables.getStackTraceAsString;
+import static com.google.common.collect.Lists.newArrayList;
+
+public class FullQueryLogTool
+{
+    public static void main(String... args)
+    {
+        DatabaseDescriptor.clientInitialization();
+        List<Class<? extends Runnable>> commands = newArrayList(
+                Help.class,
+                Dump.class,
+                Replay.class,
+                Compare.class
+        );
+
+        Cli.CliBuilder<Runnable> builder = Cli.builder("fqltool");
+
+        builder.withDescription("Manipulate the contents of full query log files")
+                 .withDefaultCommand(Help.class)
+                 .withCommands(commands);
+
+        Cli<Runnable> parser = builder.build();
+
+        int status = 0;
+        try
+        {
+            parser.parse(args).run();
+        } catch (IllegalArgumentException |
+                IllegalStateException |
+                ParseArgumentsMissingException |
+                ParseArgumentsUnexpectedException |
+                ParseOptionConversionException |
+                ParseOptionMissingException |
+                ParseOptionMissingValueException |
+                ParseCommandMissingException |
+                ParseCommandUnrecognizedException e)
+        {
+            badUse(e);
+            status = 1;
+        } catch (Throwable throwable)
+        {
+            err(Throwables.getRootCause(throwable));
+            status = 2;
+        }
+
+        System.exit(status);
+    }
+
+    private static void badUse(Exception e)
+    {
+        System.out.println("fqltool: " + e.getMessage());
+        System.out.println("See 'fqltool help' or 'fqltool help <command>'.");
+    }
+
+    private static void err(Throwable e)
+    {
+        System.err.println("error: " + e.getMessage());
+        System.err.println("-- StackTrace --");
+        System.err.println(getStackTraceAsString(e));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/QueryReplayer.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/QueryReplayer.java b/tools/fqltool/src/org/apache/cassandra/fqltool/QueryReplayer.java
new file mode 100644
index 0000000..d8653e5
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/QueryReplayer.java
@@ -0,0 +1,172 @@
+/*
+ * 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.fqltool;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+//import javax.annotation.Nullable;
+
+import com.google.common.util.concurrent.FluentFuture;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class QueryReplayer implements Closeable
+{
+    private static final int PRINT_RATE = 5000;
+    private final ExecutorService es = Executors.newFixedThreadPool(1);
+    private final Iterator<List<FQLQuery>> queryIterator;
+    private final List<Cluster> targetClusters;
+    private final List<Predicate<FQLQuery>> filters;
+    private final List<Session> sessions;
+    private final ResultHandler resultHandler;
+    private final MetricRegistry metrics = new MetricRegistry();
+    private final boolean debug;
+    private final PrintStream out;
+
+    public QueryReplayer(Iterator<List<FQLQuery>> queryIterator,
+                         List<String> targetHosts,
+                         List<File> resultPaths,
+                         List<Predicate<FQLQuery>> filters,
+                         PrintStream out,
+                         String queryFilePathString,
+                         boolean debug)
+    {
+        this.queryIterator = queryIterator;
+        targetClusters = targetHosts.stream().map(h -> Cluster.builder().addContactPoint(h).build()).collect(Collectors.toList());
+        this.filters = filters;
+        sessions = targetClusters.stream().map(Cluster::connect).collect(Collectors.toList());
+        File queryFilePath = queryFilePathString != null ? new File(queryFilePathString) : null;
+        resultHandler = new ResultHandler(targetHosts, resultPaths, queryFilePath);
+        this.debug = debug;
+        this.out = out;
+    }
+
+    public void replay()
+    {
+        while (queryIterator.hasNext())
+        {
+            List<FQLQuery> queries = queryIterator.next();
+            for (FQLQuery query : queries)
+            {
+                if (filters.stream().anyMatch(f -> !f.test(query)))
+                    continue;
+                try (Timer.Context ctx = metrics.timer("queries").time())
+                {
+                    List<ListenableFuture<ResultHandler.ComparableResultSet>> results = new ArrayList<>(sessions.size());
+                    Statement statement = query.toStatement();
+                    for (Session session : sessions)
+                    {
+                        maybeSetKeyspace(session, query);
+                        if (debug)
+                        {
+                            out.println("Executing query:");
+                            out.println(query);
+                        }
+                        ListenableFuture<ResultSet> future = session.executeAsync(statement);
+                        results.add(handleErrors(future));
+                    }
+
+                    ListenableFuture<List<ResultHandler.ComparableResultSet>> resultList = Futures.allAsList(results);
+
+                    Futures.addCallback(resultList, new FutureCallback<List<ResultHandler.ComparableResultSet>>()
+                    {
+                        public void onSuccess(/*@Nullable */List<ResultHandler.ComparableResultSet> resultSets)
+                        {
+                            // note that the order of resultSets is signifcant here - resultSets.get(x) should
+                            // be the result from a query against targetHosts.get(x)
+                            resultHandler.handleResults(query, resultSets);
+                        }
+
+                        public void onFailure(Throwable throwable)
+                        {
+                            throw new AssertionError("Errors should be handled in FQLQuery.execute", throwable);
+                        }
+                    }, es);
+
+                    FBUtilities.waitOnFuture(resultList);
+                }
+                catch (Throwable t)
+                {
+                    out.printf("QUERY %s got exception: %s", query, t.getMessage());
+                }
+
+                Timer timer = metrics.timer("queries");
+                if (timer.getCount() % PRINT_RATE == 0)
+                    out.printf("%d queries, rate = %.2f%n", timer.getCount(), timer.getOneMinuteRate());
+            }
+        }
+    }
+
+    private void maybeSetKeyspace(Session session, FQLQuery query)
+    {
+        try
+        {
+            if (query.keyspace() != null && !query.keyspace().equals(session.getLoggedKeyspace()))
+            {
+                if (debug)
+                    out.printf("Switching keyspace from %s to %s%n", session.getLoggedKeyspace(), query.keyspace());
+                session.execute("USE " + query.keyspace());
+            }
+        }
+        catch (Throwable t)
+        {
+            out.printf("USE %s failed: %s%n", query.keyspace(), t.getMessage());
+        }
+    }
+
+    /**
+     * Make sure we catch any query errors
+     *
+     * On error, this creates a failed ComparableResultSet with the exception set to be able to store
+     * this fact in the result file and handle comparison of failed result sets.
+     */
+    private static ListenableFuture<ResultHandler.ComparableResultSet> handleErrors(ListenableFuture<ResultSet> result)
+    {
+        FluentFuture<ResultHandler.ComparableResultSet> fluentFuture = FluentFuture.from(result)
+                                                                                   .transform(DriverResultSet::new, MoreExecutors.directExecutor());
+        return fluentFuture.catching(Throwable.class, DriverResultSet::failed, MoreExecutors.directExecutor());
+    }
+
+    public void close() throws IOException
+    {
+        sessions.forEach(Session::close);
+        targetClusters.forEach(Cluster::close);
+        resultHandler.close();
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[4/7] cassandra git commit: Add fqltool compare

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/test/unit/org/apache/cassandra/fqltool/FQLReplayTest.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/test/unit/org/apache/cassandra/fqltool/FQLReplayTest.java b/tools/fqltool/test/unit/org/apache/cassandra/fqltool/FQLReplayTest.java
new file mode 100644
index 0000000..61c8aa0
--- /dev/null
+++ b/tools/fqltool/test/unit/org/apache/cassandra/fqltool/FQLReplayTest.java
@@ -0,0 +1,675 @@
+/*
+ * 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.fqltool;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import com.datastax.driver.core.CodecRegistry;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.Statement;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+import net.openhft.chronicle.queue.ExcerptAppender;
+import net.openhft.chronicle.queue.ExcerptTailer;
+import org.apache.cassandra.audit.FullQueryLogger;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.fqltool.commands.Compare;
+import org.apache.cassandra.fqltool.commands.Replay;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.tools.Util;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.MergeIterator;
+import org.apache.cassandra.utils.Pair;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class FQLReplayTest
+{
+    public FQLReplayTest()
+    {
+        Util.initDatabaseDescriptor();
+    }
+
+    @Test
+    public void testOrderedReplay() throws IOException
+    {
+        File f = generateQueries(100, true);
+        int queryCount = 0;
+        try (ChronicleQueue queue = ChronicleQueueBuilder.single(f).build();
+             FQLQueryIterator iter = new FQLQueryIterator(queue.createTailer(), 101))
+        {
+            long last = -1;
+            while (iter.hasNext())
+            {
+                FQLQuery q = iter.next();
+                assertTrue(q.queryStartTime >= last);
+                last = q.queryStartTime;
+                queryCount++;
+            }
+        }
+        assertEquals(100, queryCount);
+    }
+
+    @Test
+    public void testQueryIterator() throws IOException
+    {
+        File f = generateQueries(100, false);
+        int queryCount = 0;
+        try (ChronicleQueue queue = ChronicleQueueBuilder.single(f).build();
+             FQLQueryIterator iter = new FQLQueryIterator(queue.createTailer(), 1))
+        {
+            long last = -1;
+            while (iter.hasNext())
+            {
+                FQLQuery q = iter.next();
+                assertTrue(q.queryStartTime >= last);
+                last = q.queryStartTime;
+                queryCount++;
+            }
+        }
+        assertEquals(100, queryCount);
+    }
+
+    @Test
+    public void testMergingIterator() throws IOException
+    {
+        File f = generateQueries(100, false);
+        File f2 = generateQueries(100, false);
+        int queryCount = 0;
+        try (ChronicleQueue queue = ChronicleQueueBuilder.single(f).build();
+             ChronicleQueue queue2 = ChronicleQueueBuilder.single(f2).build();
+             FQLQueryIterator iter = new FQLQueryIterator(queue.createTailer(), 101);
+             FQLQueryIterator iter2 = new FQLQueryIterator(queue2.createTailer(), 101);
+             MergeIterator<FQLQuery, List<FQLQuery>> merger = MergeIterator.get(Lists.newArrayList(iter, iter2), FQLQuery::compareTo, new Replay.Reducer()))
+        {
+            long last = -1;
+
+            while (merger.hasNext())
+            {
+                List<FQLQuery> qs = merger.next();
+                assertEquals(2, qs.size());
+                assertEquals(0, qs.get(0).compareTo(qs.get(1)));
+                assertTrue(qs.get(0).queryStartTime >= last);
+                last = qs.get(0).queryStartTime;
+                queryCount++;
+            }
+        }
+        assertEquals(100, queryCount);
+    }
+
+    @Test
+    public void testFQLQueryReader() throws IOException
+    {
+        FQLQueryReader reader = new FQLQueryReader();
+
+        try (ChronicleQueue queue = ChronicleQueueBuilder.single(generateQueries(1000, true)).build())
+        {
+            ExcerptTailer tailer = queue.createTailer();
+            int queryCount = 0;
+            while (tailer.readDocument(reader))
+            {
+                assertNotNull(reader.getQuery());
+                if (reader.getQuery() instanceof FQLQuery.Single)
+                {
+                    assertTrue(reader.getQuery().keyspace() == null || reader.getQuery().keyspace().equals("querykeyspace"));
+                }
+                else
+                {
+                    assertEquals("someks", reader.getQuery().keyspace());
+                }
+                queryCount++;
+            }
+            assertEquals(1000, queryCount);
+        }
+    }
+
+    @Test
+    public void testStoringResults() throws Throwable
+    {
+        File tmpDir = Files.createTempDirectory("results").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, true);
+        ResultStore rs = new ResultStore(Collections.singletonList(tmpDir), queryDir);
+        FQLQuery query = new FQLQuery.Single("abc", QueryOptions.DEFAULT.getProtocolVersion().asInt(), QueryOptions.DEFAULT, 12345, 11111, 22, "select * from abc", Collections.emptyList());
+        try
+        {
+            rs.storeColumnDefinitions(query, Collections.singletonList(res.getColumnDefinitions()));
+            Iterator<ResultHandler.ComparableRow> it = res.iterator();
+            while (it.hasNext())
+            {
+                List<ResultHandler.ComparableRow> row = Collections.singletonList(it.next());
+                rs.storeRows(row);
+            }
+            // this marks the end of the result set:
+            rs.storeRows(Collections.singletonList(null));
+        }
+        finally
+        {
+            rs.close();
+        }
+
+        compareResults(Collections.singletonList(Pair.create(query, res)),
+                       readResultFile(tmpDir, queryDir));
+
+    }
+
+    private static void compareResults(List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> expected,
+                                List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> other)
+    {
+        ResultComparator comparator = new ResultComparator();
+        assertEquals(expected.size(), other.size());
+        for (int i = 0; i < expected.size(); i++)
+        {
+            assertEquals(expected.get(i).left, other.get(i).left);
+            ResultHandler.ComparableResultSet expectedResultSet = expected.get(i).right;
+            ResultHandler.ComparableResultSet otherResultSet = other.get(i).right;
+            List<String> hosts = Lists.newArrayList("a", "b");
+            comparator.compareColumnDefinitions(hosts,
+                                                expected.get(i).left,
+                                                Lists.newArrayList(expectedResultSet.getColumnDefinitions(),
+                                                                   otherResultSet.getColumnDefinitions()));
+            Iterator<ResultHandler.ComparableRow> expectedRowIter = expectedResultSet.iterator();
+            Iterator<ResultHandler.ComparableRow> otherRowIter = otherResultSet.iterator();
+
+
+            while(expectedRowIter.hasNext() && otherRowIter.hasNext())
+            {
+                ResultHandler.ComparableRow expectedRow = expectedRowIter.next();
+                ResultHandler.ComparableRow otherRow = otherRowIter.next();
+                assertTrue(comparator.compareRows(hosts, expected.get(i).left, Lists.newArrayList(expectedRow, otherRow)));
+            }
+            assertFalse(expectedRowIter.hasNext());
+            assertFalse(otherRowIter.hasNext());
+        }
+    }
+
+    @Test
+    public void testCompareColumnDefinitions()
+    {
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultComparator rc = new ResultComparator();
+
+        List<ResultHandler.ComparableColumnDefinitions> colDefs = new ArrayList<>(100);
+        List<String> targetHosts = new ArrayList<>(100);
+        for (int i = 0; i < 100; i++)
+        {
+            targetHosts.add("host"+i);
+            colDefs.add(res.getColumnDefinitions());
+        }
+        assertTrue(rc.compareColumnDefinitions(targetHosts, null, colDefs));
+        colDefs.set(50, createResultSet(9, 9, false).getColumnDefinitions());
+        assertFalse(rc.compareColumnDefinitions(targetHosts, null, colDefs));
+    }
+
+    @Test
+    public void testCompareEqualRows()
+    {
+        ResultComparator rc = new ResultComparator();
+
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2);
+        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
+
+        while (true)
+        {
+            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
+            assertTrue(rc.compareRows(Lists.newArrayList("eq1", "eq2"), null, rows));
+            if (rows.stream().allMatch(Objects::isNull))
+                break;
+        }
+    }
+
+    @Test
+    public void testCompareRowsDifferentCount()
+    {
+        ResultComparator rc = new ResultComparator();
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(10, 11, false));
+        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
+        boolean foundMismatch = false;
+        while (true)
+        {
+            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
+            if (rows.stream().allMatch(Objects::isNull))
+                break;
+            if (!rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows))
+            {
+                foundMismatch = true;
+            }
+        }
+        assertTrue(foundMismatch);
+    }
+
+    @Test
+    public void testCompareRowsDifferentContent()
+    {
+        ResultComparator rc = new ResultComparator();
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(10, 10, true));
+        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
+        while (true)
+        {
+            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
+            if (rows.stream().allMatch(Objects::isNull))
+                break;
+            assertFalse(rows.toString(), rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows));
+        }
+    }
+
+    @Test
+    public void testCompareRowsDifferentColumnCount()
+    {
+        ResultComparator rc = new ResultComparator();
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(11, 10, false));
+        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
+        while (true)
+        {
+            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
+            if (rows.stream().allMatch(Objects::isNull))
+                break;
+            assertFalse(rows.toString(), rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows));
+        }
+    }
+
+    @Test
+    public void testResultHandler() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
+
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res3 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, res3);
+        FQLQuery query = new FQLQuery.Single("abcabc", QueryOptions.DEFAULT.getProtocolVersion().asInt(), QueryOptions.DEFAULT, 1111, 2222, 3333, "select * from xyz", Collections.emptyList());
+        try (ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir))
+        {
+            rh.handleResults(query, toCompare);
+        }
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(resultPaths.get(0), queryDir);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results2 = readResultFile(resultPaths.get(1), queryDir);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results3 = readResultFile(resultPaths.get(2), queryDir);
+        compareResults(results1, results2);
+        compareResults(results1, results3);
+        compareResults(results3, Collections.singletonList(Pair.create(query, res)));
+    }
+
+    @Test
+    public void testResultHandlerWithDifference() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
+
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 5, false);
+        ResultHandler.ComparableResultSet res3 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, res3);
+        FQLQuery query = new FQLQuery.Single("aaa", QueryOptions.DEFAULT.getProtocolVersion().asInt(), QueryOptions.DEFAULT, 123123, 11111, 22222, "select * from abcabc", Collections.emptyList());
+        try (ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir))
+        {
+            rh.handleResults(query, toCompare);
+        }
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(resultPaths.get(0), queryDir);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results2 = readResultFile(resultPaths.get(1), queryDir);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results3 = readResultFile(resultPaths.get(2), queryDir);
+        compareResults(results1, results3);
+        compareResults(results2, Collections.singletonList(Pair.create(query, res2)));
+    }
+
+    @Test
+    public void testResultHandlerMultipleResultSets() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
+        List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets = new ArrayList<>();
+        Random random = new Random();
+        for (int i = 0; i < 10; i++)
+        {
+            List<ResultHandler.ComparableResultSet> results = new ArrayList<>();
+            List<ByteBuffer> values = Collections.singletonList(ByteBufferUtil.bytes(i * 50));
+            for (int jj = 0; jj < targetHosts.size(); jj++)
+            {
+                results.add(createResultSet(5, 1 + random.nextInt(10), true));
+            }
+            FQLQuery q = i % 2 == 0
+                         ? new FQLQuery.Single("abc"+i,
+                                             3,
+                                             QueryOptions.forInternalCalls(values),
+                                             i * 1000,
+                                             12345,
+                                             54321,
+                                             "select * from xyz where id = "+i,
+                                             values)
+                         : new FQLQuery.Batch("abc"+i,
+                                              3,
+                                              QueryOptions.forInternalCalls(values),
+                                              i * 1000,
+                                              i * 54321,
+                                              i * 12345,
+                                              com.datastax.driver.core.BatchStatement.Type.UNLOGGED,
+                                              Lists.newArrayList("select * from aaaa"),
+                                              Collections.singletonList(values));
+
+            resultSets.add(Pair.create(q, results));
+        }
+        try (ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir))
+        {
+            for (int i = 0; i < resultSets.size(); i++)
+                rh.handleResults(resultSets.get(i).left, resultSets.get(i).right);
+        }
+
+        for (int i = 0; i < targetHosts.size(); i++)
+            compareWithFile(resultPaths, queryDir, resultSets, i);
+    }
+
+    @Test
+    public void testResultHandlerFailedQuery() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc", "hostd");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
+
+        List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets = new ArrayList<>();
+        Random random = new Random();
+        for (int i = 0; i < 10; i++)
+        {
+            List<ResultHandler.ComparableResultSet> results = new ArrayList<>();
+            List<ByteBuffer> values = Collections.singletonList(ByteBufferUtil.bytes(i * 50));
+            for (int jj = 0; jj < targetHosts.size(); jj++)
+            {
+                results.add(createResultSet(5, 1 + random.nextInt(10), true));
+            }
+            results.set(0, StoredResultSet.failed("testing abc"));
+            results.set(3, StoredResultSet.failed("testing abc"));
+            FQLQuery q = new FQLQuery.Single("abc"+i,
+                                             3,
+                                             QueryOptions.forInternalCalls(values),
+                                             i * 1000,
+                                             i * 12345,
+                                             i * 54321,
+                                             "select * from xyz where id = "+i,
+                                             values);
+            resultSets.add(Pair.create(q, results));
+        }
+        try (ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir))
+        {
+            for (int i = 0; i < resultSets.size(); i++)
+                rh.handleResults(resultSets.get(i).left, resultSets.get(i).right);
+        }
+        for (int i = 0; i < targetHosts.size(); i++)
+            compareWithFile(resultPaths, queryDir, resultSets, i);
+    }
+
+    @Test
+    public void testCompare()
+    {
+        FQLQuery q1 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.emptyList());
+        FQLQuery q2 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222,"aaaa", Collections.emptyList());
+
+        assertEquals(0, q1.compareTo(q2));
+        assertEquals(0, q2.compareTo(q1));
+
+        FQLQuery q3 = new FQLQuery.Batch("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, com.datastax.driver.core.BatchStatement.Type.UNLOGGED, Collections.emptyList(), Collections.emptyList());
+        // single queries before batch queries
+        assertTrue(q1.compareTo(q3) < 0);
+        assertTrue(q3.compareTo(q1) > 0);
+
+        // check that smaller query time
+        FQLQuery q4 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 124, 111, 222, "aaaa", Collections.emptyList());
+        assertTrue(q1.compareTo(q4) < 0);
+        assertTrue(q4.compareTo(q1) > 0);
+
+        FQLQuery q5 = new FQLQuery.Batch("abc", 0, QueryOptions.DEFAULT, 124, 111, 222, com.datastax.driver.core.BatchStatement.Type.UNLOGGED, Collections.emptyList(), Collections.emptyList());
+        assertTrue(q1.compareTo(q5) < 0);
+        assertTrue(q5.compareTo(q1) > 0);
+
+        FQLQuery q6 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes(10)));
+        FQLQuery q7 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.emptyList());
+        assertTrue(q6.compareTo(q7) > 0);
+        assertTrue(q7.compareTo(q6) < 0);
+
+        FQLQuery q8 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes("a")));
+        FQLQuery q9 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes("b")));
+        assertTrue(q8.compareTo(q9) < 0);
+        assertTrue(q9.compareTo(q8) > 0);
+    }
+
+    @Test
+    public void testFQLQuerySingleToStatement()
+    {
+        List<ByteBuffer> values = new ArrayList<>();
+        for (int i = 0; i < 10; i++)
+            values.add(ByteBufferUtil.bytes(i));
+        FQLQuery.Single single = new FQLQuery.Single("xyz",
+                                                     QueryOptions.DEFAULT.getProtocolVersion().asInt(),
+                                                     QueryOptions.forInternalCalls(values),
+                                                     1234,
+                                                     12345,
+                                                     54321,
+                                                     "select * from aaa",
+                                                     values);
+        Statement stmt = single.toStatement();
+        assertEquals(stmt.getDefaultTimestamp(), 12345);
+        assertTrue(stmt instanceof SimpleStatement);
+        SimpleStatement simpleStmt = (SimpleStatement)stmt;
+        assertEquals("select * from aaa",simpleStmt.getQueryString(CodecRegistry.DEFAULT_INSTANCE));
+        assertArrayEquals(values.toArray(), simpleStmt.getValues(com.datastax.driver.core.ProtocolVersion.fromInt(QueryOptions.DEFAULT.getProtocolVersion().asInt()), CodecRegistry.DEFAULT_INSTANCE));
+    }
+
+
+    @Test
+    public void testFQLQueryBatchToStatement()
+    {
+        List<List<ByteBuffer>> values = new ArrayList<>();
+        List<String> queries = new ArrayList<>();
+        for (int bqCount = 0; bqCount < 10; bqCount++)
+        {
+            queries.add("select * from asdf where x = ? and y = " + bqCount);
+            List<ByteBuffer> queryValues = new ArrayList<>();
+            for (int i = 0; i < 10; i++)
+                queryValues.add(ByteBufferUtil.bytes(i + ":" + bqCount));
+            values.add(queryValues);
+        }
+
+        FQLQuery.Batch batch = new FQLQuery.Batch("xyz",
+                                                   QueryOptions.DEFAULT.getProtocolVersion().asInt(),
+                                                   QueryOptions.DEFAULT,
+                                                   1234,
+                                                   12345,
+                                                   54321,
+                                                   com.datastax.driver.core.BatchStatement.Type.UNLOGGED,
+                                                   queries,
+                                                   values);
+        Statement stmt = batch.toStatement();
+        assertEquals(stmt.getDefaultTimestamp(), 12345);
+        assertTrue(stmt instanceof com.datastax.driver.core.BatchStatement);
+        com.datastax.driver.core.BatchStatement batchStmt = (com.datastax.driver.core.BatchStatement)stmt;
+        List<Statement> statements = Lists.newArrayList(batchStmt.getStatements());
+        List<Statement> fromFQLQueries = batch.queries.stream().map(FQLQuery.Single::toStatement).collect(Collectors.toList());
+        assertEquals(statements.size(), fromFQLQueries.size());
+        assertEquals(12345, batchStmt.getDefaultTimestamp());
+        for (int i = 0; i < statements.size(); i++)
+            compareStatements(statements.get(i), fromFQLQueries.get(i));
+    }
+
+    private void compareStatements(Statement statement1, Statement statement2)
+    {
+        assertTrue(statement1 instanceof SimpleStatement && statement2 instanceof SimpleStatement);
+        SimpleStatement simpleStmt1 = (SimpleStatement)statement1;
+        SimpleStatement simpleStmt2 = (SimpleStatement)statement2;
+        assertEquals(simpleStmt1.getQueryString(CodecRegistry.DEFAULT_INSTANCE), simpleStmt2.getQueryString(CodecRegistry.DEFAULT_INSTANCE));
+        assertArrayEquals(simpleStmt1.getValues(com.datastax.driver.core.ProtocolVersion.fromInt(QueryOptions.DEFAULT.getProtocolVersion().asInt()), CodecRegistry.DEFAULT_INSTANCE),
+                          simpleStmt2.getValues(com.datastax.driver.core.ProtocolVersion.fromInt(QueryOptions.DEFAULT.getProtocolVersion().asInt()), CodecRegistry.DEFAULT_INSTANCE));
+
+    }
+
+    private File generateQueries(int count, boolean random) throws IOException
+    {
+        Random r = new Random();
+        File dir = Files.createTempDirectory("chronicle").toFile();
+        try (ChronicleQueue readQueue = ChronicleQueueBuilder.single(dir).build())
+        {
+            ExcerptAppender appender = readQueue.acquireAppender();
+
+            for (int i = 0; i < count; i++)
+            {
+                long timestamp = random ? Math.abs(r.nextLong() % 10000) : i;
+                if (random ? r.nextBoolean() : i % 2 == 0)
+                {
+                    String query = "abcdefghijklm " + i;
+                    QueryState qs = r.nextBoolean() ? queryState() : queryState("querykeyspace");
+                    FullQueryLogger.Query  q = new FullQueryLogger.Query(query, QueryOptions.DEFAULT, qs, timestamp);
+                    appender.writeDocument(q);
+                    q.release();
+                }
+                else
+                {
+                    int batchSize = random ? r.nextInt(99) + 1 : i + 1;
+                    List<String> queries = new ArrayList<>(batchSize);
+                    List<List<ByteBuffer>> values = new ArrayList<>(batchSize);
+                    for (int jj = 0; jj < (random ? r.nextInt(batchSize) : 10); jj++)
+                    {
+                        queries.add("aaaaaa batch "+i+":"+jj);
+                        values.add(Collections.emptyList());
+                    }
+                    FullQueryLogger.Batch batch = new FullQueryLogger.Batch(BatchStatement.Type.UNLOGGED,
+                                                                            queries,
+                                                                            values,
+                                                                            QueryOptions.DEFAULT,
+                                                                            queryState("someks"),
+                                                                            timestamp);
+                    appender.writeDocument(batch);
+                    batch.release();
+                }
+            }
+        }
+        return dir;
+    }
+
+    private QueryState queryState()
+    {
+        return QueryState.forInternalCalls();
+    }
+
+    private QueryState queryState(String keyspace)
+    {
+        ClientState clientState = ClientState.forInternalCalls(keyspace);
+        return new QueryState(clientState);
+    }
+
+    static ResultHandler.ComparableResultSet createResultSet(int columnCount, int rowCount, boolean random)
+    {
+        List<Pair<String, String>> columnDefs = new ArrayList<>(columnCount);
+        Random r = new Random();
+        for (int i = 0; i < columnCount; i++)
+        {
+            columnDefs.add(Pair.create("a" + i, "int"));
+        }
+        ResultHandler.ComparableColumnDefinitions colDefs = new StoredResultSet.StoredComparableColumnDefinitions(columnDefs, false, null);
+        List<ResultHandler.ComparableRow> rows = new ArrayList<>();
+        for (int i = 0; i < rowCount; i++)
+        {
+            List<ByteBuffer> row = new ArrayList<>(columnCount);
+            for (int jj = 0; jj < columnCount; jj++)
+                row.add(ByteBufferUtil.bytes(i + " col " + jj + (random ? r.nextInt() : "")));
+
+            rows.add(new StoredResultSet.StoredComparableRow(row, colDefs));
+        }
+        return new StoredResultSet(colDefs, true, false, null, rows::iterator);
+    }
+
+    private static void compareWithFile(List<File> dirs, File resultDir, List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets, int idx)
+    {
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(dirs.get(idx), resultDir);
+        for (int i = 0; i < resultSets.size(); i++)
+        {
+            FQLQuery query = resultSets.get(i).left;
+            List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> toCompare = Collections.singletonList(Pair.create(query, resultSets.get(i).right.get(idx)));
+            compareResults(Collections.singletonList(results1.get(i)), toCompare);
+        }
+    }
+
+    private static List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> readResultFile(File dir, File queryDir)
+    {
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> resultSets = new ArrayList<>();
+        try (ChronicleQueue q = ChronicleQueueBuilder.single(dir).build();
+             ChronicleQueue queryQ = ChronicleQueueBuilder.single(queryDir).build())
+        {
+            ExcerptTailer queryTailer = queryQ.createTailer();
+            FQLQueryReader queryReader = new FQLQueryReader();
+            Compare.StoredResultSetIterator resultSetIterator = new Compare.StoredResultSetIterator(q.createTailer());
+            // we need to materialize the rows in-memory to compare them easier in these tests
+            while (resultSetIterator.hasNext())
+            {
+                ResultHandler.ComparableResultSet resultSetFromDisk = resultSetIterator.next();
+                Iterator<ResultHandler.ComparableRow> rowIterFromDisk = resultSetFromDisk.iterator();
+                queryTailer.readDocument(queryReader);
+
+                FQLQuery query = queryReader.getQuery();
+                List<ResultHandler.ComparableRow> rows = new ArrayList<>();
+                while (rowIterFromDisk.hasNext())
+                {
+                    rows.add(rowIterFromDisk.next());
+                }
+                resultSets.add(Pair.create(query, new StoredResultSet(resultSetFromDisk.getColumnDefinitions(),
+                                                                      resultSetIterator.hasNext(),
+                                                                      resultSetFromDisk.wasFailed(),
+                                                                      resultSetFromDisk.getFailureException(),
+                                                                      rows::iterator)));
+            }
+        }
+        return resultSets;
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[5/7] cassandra git commit: Add fqltool compare

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/ResultComparator.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/ResultComparator.java b/tools/fqltool/src/org/apache/cassandra/fqltool/ResultComparator.java
new file mode 100644
index 0000000..d8d419a
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/ResultComparator.java
@@ -0,0 +1,116 @@
+/*
+ * 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.fqltool;
+
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Streams;
+
+public class ResultComparator
+{
+    /**
+     * Compares the rows in rows
+     * the row at position x in rows will have come from host at position x in targetHosts
+     */
+    public boolean compareRows(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableRow> rows)
+    {
+        if (rows.size() < 2 || rows.stream().allMatch(Objects::isNull))
+            return true;
+
+        if (rows.stream().anyMatch(Objects::isNull))
+        {
+            handleMismatch(targetHosts, query, rows);
+            return false;
+        }
+
+        ResultHandler.ComparableRow ref = rows.get(0);
+        boolean equal = true;
+        for (int i = 1; i < rows.size(); i++)
+        {
+            ResultHandler.ComparableRow compare = rows.get(i);
+            if (!ref.equals(compare))
+                equal = false;
+        }
+        if (!equal)
+            handleMismatch(targetHosts, query, rows);
+        return equal;
+    }
+
+    /**
+     * Compares the column definitions
+     *
+     * the column definitions at position x in cds will have come from host at position x in targetHosts
+     */
+    public boolean compareColumnDefinitions(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
+    {
+        if (cds.size() < 2)
+            return true;
+
+        boolean equal = true;
+        List<ResultHandler.ComparableDefinition> refDefs = cds.get(0).asList();
+        for (int i = 1; i < cds.size(); i++)
+        {
+            List<ResultHandler.ComparableDefinition> toCompare = cds.get(i).asList();
+            if (!refDefs.equals(toCompare))
+                equal = false;
+        }
+        if (!equal)
+            handleColumnDefMismatch(targetHosts, query, cds);
+        return equal;
+    }
+
+    private void handleMismatch(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableRow> rows)
+    {
+        System.out.println("MISMATCH:");
+        System.out.println("Query = " + query);
+        System.out.println("Results:");
+        System.out.println(Streams.zip(rows.stream(), targetHosts.stream(), (r, host) -> String.format("%s: %s%n", host, r == null ? "null" : r)).collect(Collectors.joining()));
+    }
+
+    private void handleColumnDefMismatch(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
+    {
+        System.out.println("COLUMN DEFINITION MISMATCH:");
+        System.out.println("Query = " + query);
+        System.out.println("Results: ");
+        System.out.println(Streams.zip(cds.stream(), targetHosts.stream(), (cd, host) -> String.format("%s: %s%n", host, columnDefinitionsString(cd))).collect(Collectors.joining()));
+    }
+
+    private String columnDefinitionsString(ResultHandler.ComparableColumnDefinitions cd)
+    {
+        StringBuilder sb = new StringBuilder();
+        if (cd == null)
+            sb.append("NULL");
+        else if (cd.wasFailed())
+            sb.append("FAILED");
+        else
+        {
+            for (ResultHandler.ComparableDefinition def : cd)
+            {
+                sb.append(def.toString());
+            }
+        }
+        return sb.toString();
+    }
+
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/ResultHandler.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/ResultHandler.java b/tools/fqltool/src/org/apache/cassandra/fqltool/ResultHandler.java
new file mode 100644
index 0000000..8c4c018
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/ResultHandler.java
@@ -0,0 +1,133 @@
+/*
+ * 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.fqltool;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class ResultHandler implements Closeable
+{
+    private final ResultStore resultStore;
+    private final ResultComparator resultComparator;
+    private final List<String> targetHosts;
+
+    public ResultHandler(List<String> targetHosts, List<File> resultPaths, File queryFilePath)
+    {
+        this.targetHosts = targetHosts;
+        resultStore = resultPaths != null ? new ResultStore(resultPaths, queryFilePath) : null;
+        resultComparator = new ResultComparator();
+    }
+
+    /**
+     * Since we can't iterate a ResultSet more than once, and we don't want to keep the entire result set in memory
+     * we feed the rows one-by-one to resultComparator and resultStore.
+     *
+     * results.get(x) should be the results from executing query against targetHosts.get(x)
+     */
+    public void handleResults(FQLQuery query, List<ComparableResultSet> results)
+    {
+        for (int i = 0; i < targetHosts.size(); i++)
+        {
+            if (results.get(i).wasFailed())
+            {
+                System.out.println("Query against "+targetHosts.get(i)+" failure:");
+                System.out.println(query);
+                System.out.println("Message: "+results.get(i).getFailureException().getMessage());
+            }
+        }
+
+        List<ComparableColumnDefinitions> columnDefinitions = results.stream().map(ComparableResultSet::getColumnDefinitions).collect(Collectors.toList());
+        resultComparator.compareColumnDefinitions(targetHosts, query, columnDefinitions);
+        if (resultStore != null)
+            resultStore.storeColumnDefinitions(query, columnDefinitions);
+        List<Iterator<ComparableRow>> iters = results.stream().map(Iterable::iterator).collect(Collectors.toList());
+
+        while (true)
+        {
+            List<ComparableRow> rows = rows(iters);
+            resultComparator.compareRows(targetHosts, query, rows);
+            if (resultStore != null)
+                resultStore.storeRows(rows);
+            // all rows being null marks end of all resultsets, we need to call compareRows
+            // and storeRows once with everything null to mark that fact
+            if (rows.stream().allMatch(Objects::isNull))
+                return;
+        }
+    }
+
+    /**
+     * Get the first row from each of the iterators, if the iterator has run out, null will mark that in the list
+     */
+    @VisibleForTesting
+    public static List<ComparableRow> rows(List<Iterator<ComparableRow>> iters)
+    {
+        List<ComparableRow> rows = new ArrayList<>(iters.size());
+        for (Iterator<ComparableRow> iter : iters)
+        {
+            if (iter.hasNext())
+                rows.add(iter.next());
+            else
+                rows.add(null);
+        }
+        return rows;
+    }
+
+    public void close() throws IOException
+    {
+        if (resultStore != null)
+            resultStore.close();
+    }
+
+    public interface ComparableResultSet extends Iterable<ComparableRow>
+    {
+        public ComparableColumnDefinitions getColumnDefinitions();
+        public boolean wasFailed();
+        public Throwable getFailureException();
+    }
+
+    public interface ComparableColumnDefinitions extends Iterable<ComparableDefinition>
+    {
+        public List<ComparableDefinition> asList();
+        public boolean wasFailed();
+        public Throwable getFailureException();
+        public int size();
+    }
+
+    public interface ComparableDefinition
+    {
+        public String getType();
+        public String getName();
+    }
+
+    public interface ComparableRow
+    {
+        public ByteBuffer getBytesUnsafe(int i);
+        public ComparableColumnDefinitions getColumnDefinitions();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/ResultStore.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/ResultStore.java b/tools/fqltool/src/org/apache/cassandra/fqltool/ResultStore.java
new file mode 100644
index 0000000..d128717
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/ResultStore.java
@@ -0,0 +1,291 @@
+/*
+ * 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.fqltool;
+
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import net.openhft.chronicle.bytes.BytesStore;
+import net.openhft.chronicle.core.io.Closeable;
+import net.openhft.chronicle.core.io.IORuntimeException;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+import net.openhft.chronicle.queue.ExcerptAppender;
+import net.openhft.chronicle.wire.ReadMarshallable;
+import net.openhft.chronicle.wire.ValueIn;
+import net.openhft.chronicle.wire.ValueOut;
+import net.openhft.chronicle.wire.WireIn;
+import net.openhft.chronicle.wire.WireOut;
+import net.openhft.chronicle.wire.WriteMarshallable;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.binlog.BinLog;
+
+/**
+ * note that we store each row as a separate chronicle document to be able to
+ * avoid reading up the entire result set in memory when comparing
+ *
+ * document formats:
+ * to mark the start of a new result set:
+ * -------------------
+ * version: int16
+ * type: column_definitions
+ * column_count: int32;
+ * column_definition: text, text
+ * column_definition: text, text
+ * ....
+ * --------------------
+ *
+ * to mark a failed query:
+ * ---------------------
+ * version: int16
+ * type: query_failed
+ * message: text
+ * ---------------------
+ *
+ * row:
+ * --------------------
+ * version: int16
+ * type: row
+ * row_column_count: int32
+ * column: bytes
+ * ---------------------
+ *
+ * to mark the end of a result set:
+ * -------------------
+ * version: int16
+ * type: end_resultset
+ * -------------------
+ *
+ */
+public class ResultStore
+{
+    private static final String VERSION = "version";
+    private static final String TYPE = "type";
+    // types:
+    private static final String ROW = "row";
+    private static final String END = "end_resultset";
+    private static final String FAILURE = "query_failed";
+    private static final String COLUMN_DEFINITIONS = "column_definitions";
+    // fields:
+    private static final String COLUMN_DEFINITION = "column_definition";
+    private static final String COLUMN_COUNT = "column_count";
+    private static final String MESSAGE = "message";
+    private static final String ROW_COLUMN_COUNT = "row_column_count";
+    private static final String COLUMN = "column";
+
+    private static final int CURRENT_VERSION = 0;
+
+    private final List<ChronicleQueue> queues;
+    private final List<ExcerptAppender> appenders;
+    private final ChronicleQueue queryStoreQueue;
+    private final ExcerptAppender queryStoreAppender;
+    private final Set<Integer> finishedHosts = new HashSet<>();
+
+    public ResultStore(List<File> resultPaths, File queryFilePath)
+    {
+        queues = resultPaths.stream().map(path -> ChronicleQueueBuilder.single(path).build()).collect(Collectors.toList());
+        appenders = queues.stream().map(ChronicleQueue::acquireAppender).collect(Collectors.toList());
+        queryStoreQueue = queryFilePath != null ? ChronicleQueueBuilder.single(queryFilePath).build() : null;
+        queryStoreAppender = queryStoreQueue != null ? queryStoreQueue.acquireAppender() : null;
+    }
+
+    /**
+     * Store the column definitions in cds
+     *
+     * the ColumnDefinitions at position x will get stored by the appender at position x
+     *
+     * Calling this method indicates that we are starting a new result set from a query, it must be called before
+     * calling storeRows.
+     *
+     */
+    public void storeColumnDefinitions(FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
+    {
+        finishedHosts.clear();
+        if (queryStoreAppender != null)
+        {
+            BinLog.ReleaseableWriteMarshallable writeMarshallableQuery = query.toMarshallable();
+            queryStoreAppender.writeDocument(writeMarshallableQuery);
+            writeMarshallableQuery.release();
+        }
+        for (int i = 0; i < cds.size(); i++)
+        {
+            ResultHandler.ComparableColumnDefinitions cd = cds.get(i);
+            appenders.get(i).writeDocument(new ColumnDefsWriter(cd));
+        }
+    }
+
+    /**
+     * Store rows
+     *
+     * the row at position x will get stored by appender at position x
+     *
+     * Before calling this for a new result set, storeColumnDefinitions must be called.
+     */
+    public void storeRows(List<ResultHandler.ComparableRow> rows)
+    {
+        for (int i = 0; i < rows.size(); i++)
+        {
+            ResultHandler.ComparableRow row = rows.get(i);
+            if (row == null && !finishedHosts.contains(i))
+            {
+                appenders.get(i).writeDocument(wire -> {
+                    wire.write(VERSION).int16(CURRENT_VERSION);
+                    wire.write(TYPE).text(END);
+                });
+                finishedHosts.add(i);
+            }
+            else if (row != null)
+            {
+                appenders.get(i).writeDocument(new RowWriter(row));
+            }
+        }
+    }
+
+    public void close()
+    {
+        queues.forEach(Closeable::close);
+        if (queryStoreQueue != null)
+            queryStoreQueue.close();
+    }
+
+    static class ColumnDefsWriter implements WriteMarshallable
+    {
+        private final ResultHandler.ComparableColumnDefinitions defs;
+
+        ColumnDefsWriter(ResultHandler.ComparableColumnDefinitions defs)
+        {
+            this.defs = defs;
+        }
+
+        public void writeMarshallable(WireOut wire)
+        {
+            wire.write(VERSION).int16(CURRENT_VERSION);
+            if (!defs.wasFailed())
+            {
+                wire.write(TYPE).text(COLUMN_DEFINITIONS);
+                wire.write(COLUMN_COUNT).int32(defs.size());
+                for (ResultHandler.ComparableDefinition d : defs.asList())
+                {
+                    ValueOut vo = wire.write(COLUMN_DEFINITION);
+                    vo.text(d.getName());
+                    vo.text(d.getType());
+                }
+            }
+            else
+            {
+                wire.write(TYPE).text(FAILURE);
+                wire.write(MESSAGE).text(defs.getFailureException().getMessage());
+            }
+        }
+    }
+
+    static class ColumnDefsReader implements ReadMarshallable
+    {
+        boolean wasFailed;
+        String failureMessage;
+        List<Pair<String, String>> columnDefinitions = new ArrayList<>();
+
+        public void readMarshallable(WireIn wire) throws IORuntimeException
+        {
+            int version = wire.read(VERSION).int16();
+            String type = wire.read(TYPE).text();
+            if (type.equals(FAILURE))
+            {
+                wasFailed = true;
+                failureMessage = wire.read(MESSAGE).text();
+            }
+            else if (type.equals(COLUMN_DEFINITION))
+            {
+                int columnCount = wire.read(COLUMN_COUNT).int32();
+                for (int i = 0; i < columnCount; i++)
+                {
+                    ValueIn vi = wire.read(COLUMN_DEFINITION);
+                    String name = vi.text();
+                    String dataType = vi.text();
+                    columnDefinitions.add(Pair.create(name, dataType));
+                }
+            }
+        }
+    }
+
+    /**
+     * read a single row from the wire, or, marks itself finished if we read "end_resultset"
+     */
+    static class RowReader implements ReadMarshallable
+    {
+        boolean isFinished;
+        List<ByteBuffer> rows = new ArrayList<>();
+
+        public void readMarshallable(WireIn wire) throws IORuntimeException
+        {
+            int version = wire.read(VERSION).int32();
+            String type = wire.read(TYPE).text();
+            if (!type.equals(END))
+            {
+                isFinished = false;
+                int rowColumnCount = wire.read(ROW_COLUMN_COUNT).int32();
+
+                for (int i = 0; i < rowColumnCount; i++)
+                {
+                    byte[] b = wire.read(COLUMN).bytes();
+                    rows.add(ByteBuffer.wrap(b));
+                }
+            }
+            else
+            {
+                isFinished = true;
+            }
+        }
+    }
+
+    /**
+     * Writes a single row to the given wire
+     */
+    static class RowWriter implements WriteMarshallable
+    {
+        private final ResultHandler.ComparableRow row;
+
+        RowWriter(ResultHandler.ComparableRow row)
+        {
+            this.row = row;
+        }
+
+        public void writeMarshallable(WireOut wire)
+        {
+            wire.write(VERSION).int16(CURRENT_VERSION);
+            wire.write(TYPE).text(ROW);
+            wire.write(ROW_COLUMN_COUNT).int32(row.getColumnDefinitions().size());
+            for (int jj = 0; jj < row.getColumnDefinitions().size(); jj++)
+            {
+                ByteBuffer bb = row.getBytesUnsafe(jj);
+                if (bb != null)
+                    wire.write(COLUMN).bytes(BytesStore.wrap(bb));
+                else
+                    wire.write(COLUMN).bytes("NULL".getBytes());
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/StoredResultSet.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/StoredResultSet.java b/tools/fqltool/src/org/apache/cassandra/fqltool/StoredResultSet.java
new file mode 100644
index 0000000..b08861d
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/StoredResultSet.java
@@ -0,0 +1,292 @@
+/*
+ * 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.fqltool;
+
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.AbstractIterator;
+
+import net.openhft.chronicle.queue.ExcerptTailer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * represents a resultset defined by the format in ResultStore on disk
+ *
+ * todo: Currently all iterators need to be consumed fully while iterating over result sets
+ *       if this is created from a tailer. This can probably be improved, but for all current uses it is fine.
+ */
+public class StoredResultSet implements ResultHandler.ComparableResultSet
+{
+    private final ResultHandler.ComparableColumnDefinitions defs;
+    public final boolean hasMoreResultSets;
+    private final Supplier<Iterator<ResultHandler.ComparableRow>> rowIteratorSupplier;
+    private final boolean wasFailed;
+    private final Throwable failureException;
+
+    /**
+     * create a new StoredResultSet
+     *
+     * note that we use an iteratorSupplier to be able to iterate over the same in-memory rows several times *in tests*
+     */
+    public StoredResultSet(ResultHandler.ComparableColumnDefinitions defs,
+                           boolean hasMoreResultSets,
+                           boolean wasFailed,
+                           Throwable failure,
+                           Supplier<Iterator<ResultHandler.ComparableRow>> iteratorSupplier)
+    {
+        this.defs = defs;
+        this.hasMoreResultSets = hasMoreResultSets;
+        this.wasFailed = wasFailed;
+        this.failureException = failure;
+        this.rowIteratorSupplier = iteratorSupplier;
+    }
+
+    /**
+     * creates a ComparableResultSet based on the data in tailer
+     */
+    public static StoredResultSet fromTailer(ExcerptTailer tailer)
+    {
+        ResultStore.ColumnDefsReader reader = new ResultStore.ColumnDefsReader();
+        boolean hasMoreResultSets = tailer.readDocument(reader);
+        ResultHandler.ComparableColumnDefinitions defs = new StoredComparableColumnDefinitions(reader.columnDefinitions,
+                                                                                               reader.wasFailed,
+                                                                                               new RuntimeException(reader.failureMessage));
+
+
+        Iterator<ResultHandler.ComparableRow> rowIterator = new AbstractIterator<ResultHandler.ComparableRow>()
+        {
+            protected ResultHandler.ComparableRow computeNext()
+            {
+                ResultStore.RowReader rowReader = new ResultStore.RowReader();
+                tailer.readDocument(rowReader);
+                if (rowReader.isFinished)
+                    return endOfData();
+                return new StoredComparableRow(rowReader.rows, defs);
+            }
+        };
+
+        return new StoredResultSet(defs,
+                                   hasMoreResultSets,
+                                   reader.wasFailed,
+                                   new RuntimeException(reader.failureMessage),
+                                   () -> rowIterator);
+    }
+
+    public static ResultHandler.ComparableResultSet failed(String failureMessage)
+    {
+        return new FailedComparableResultSet(new RuntimeException(failureMessage));
+    }
+
+    public Iterator<ResultHandler.ComparableRow> iterator()
+    {
+        return rowIteratorSupplier.get();
+    }
+
+    public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+    {
+        return defs;
+    }
+
+    public boolean wasFailed()
+    {
+        return wasFailed;
+    }
+
+    public Throwable getFailureException()
+    {
+        return failureException;
+    }
+
+    static class StoredComparableRow implements ResultHandler.ComparableRow
+    {
+        private final List<ByteBuffer> row;
+        private final ResultHandler.ComparableColumnDefinitions cds;
+
+        public StoredComparableRow(List<ByteBuffer> row, ResultHandler.ComparableColumnDefinitions cds)
+        {
+            this.row = row;
+            this.cds = cds;
+        }
+
+        public ByteBuffer getBytesUnsafe(int i)
+        {
+            return row.get(i);
+        }
+
+        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+        {
+            return cds;
+        }
+
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof StoredComparableRow))
+                return false;
+            return row.equals(((StoredComparableRow)other).row);
+        }
+
+        public String toString()
+        {
+            return row.stream().map(ByteBufferUtil::bytesToHex).collect(Collectors.joining(","));
+        }
+    }
+
+    static class StoredComparableColumnDefinitions implements ResultHandler.ComparableColumnDefinitions
+    {
+        private final List<ResultHandler.ComparableDefinition> defs;
+        private final boolean wasFailed;
+        private final Throwable failureException;
+
+        public StoredComparableColumnDefinitions(List<Pair<String, String>> cds, boolean wasFailed, Throwable failureException)
+        {
+            defs = cds != null ? cds.stream().map(StoredComparableDefinition::new).collect(Collectors.toList()) : Collections.emptyList();
+            this.wasFailed = wasFailed;
+            this.failureException = failureException;
+        }
+        public List<ResultHandler.ComparableDefinition> asList()
+        {
+            return wasFailed() ? Collections.emptyList() : defs;
+        }
+
+        public boolean wasFailed()
+        {
+            return wasFailed;
+        }
+
+        public Throwable getFailureException()
+        {
+            return failureException;
+        }
+
+        public int size()
+        {
+            return asList().size();
+        }
+
+        public Iterator<ResultHandler.ComparableDefinition> iterator()
+        {
+            return defs.iterator();
+        }
+
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof StoredComparableColumnDefinitions))
+                return false;
+            return defs.equals(((StoredComparableColumnDefinitions)other).defs);
+        }
+
+        public String toString()
+        {
+            return defs.toString();
+        }
+    }
+
+    private static class StoredComparableDefinition implements ResultHandler.ComparableDefinition
+    {
+        private final Pair<String, String> p;
+
+        public StoredComparableDefinition(Pair<String, String> p)
+        {
+            this.p = p;
+        }
+        public String getType()
+        {
+            return p.right;
+        }
+
+        public String getName()
+        {
+            return p.left;
+        }
+
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof StoredComparableDefinition))
+                return false;
+            return p.equals(((StoredComparableDefinition)other).p);
+        }
+
+        public String toString()
+        {
+            return getName() + ':' + getType();
+        }
+    }
+
+    private static class FailedComparableResultSet implements ResultHandler.ComparableResultSet
+    {
+        private final Throwable exception;
+
+        public FailedComparableResultSet(Throwable exception)
+        {
+            this.exception = exception;
+        }
+        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+        {
+            return new ResultHandler.ComparableColumnDefinitions()
+            {
+                public List<ResultHandler.ComparableDefinition> asList()
+                {
+                    return Collections.emptyList();
+                }
+
+                public boolean wasFailed()
+                {
+                    return true;
+                }
+
+                public Throwable getFailureException()
+                {
+                    return exception;
+                }
+
+                public int size()
+                {
+                    return 0;
+                }
+
+                public Iterator<ResultHandler.ComparableDefinition> iterator()
+                {
+                    return asList().iterator();
+                }
+            };
+        }
+
+        public boolean wasFailed()
+        {
+            return true;
+        }
+
+        public Throwable getFailureException()
+        {
+            return new RuntimeException();
+        }
+
+        public Iterator<ResultHandler.ComparableRow> iterator()
+        {
+            return Collections.emptyListIterator();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Compare.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Compare.java b/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Compare.java
new file mode 100644
index 0000000..2375296
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Compare.java
@@ -0,0 +1,120 @@
+/*
+ * 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.fqltool.commands;
+
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.AbstractIterator;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import io.airlift.airline.Option;
+import net.openhft.chronicle.core.io.Closeable;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+import net.openhft.chronicle.queue.ExcerptTailer;
+import org.apache.cassandra.fqltool.FQLQueryIterator;
+import org.apache.cassandra.fqltool.ResultHandler;
+import org.apache.cassandra.fqltool.StoredResultSet;
+
+/**
+ */
+@Command(name = "compare", description = "Compare result files generated by fqltool replay")
+public class Compare implements Runnable
+{
+    @Arguments(usage = "<path1> [<path2>...<pathN>]",
+               description = "Directories containing result files to compare.",
+               required = true)
+    private List<String> arguments = new ArrayList<>();
+
+    @Option(title = "queries",
+            name = { "--queries"},
+            description = "Directory to read the queries from. It is produced by the fqltool replay --store-queries option. ",
+            required = true)
+    private String querylog;
+
+    @Override
+    public void run()
+    {
+        compare(querylog, arguments);
+    }
+
+    public static void compare(String querylog, List<String> arguments)
+    {
+        List<ChronicleQueue> readQueues = null;
+        try (ResultHandler rh = new ResultHandler(arguments, null, null);
+             ChronicleQueue queryQ = ChronicleQueueBuilder.single(querylog).readOnly(true).build();
+             FQLQueryIterator queries = new FQLQueryIterator(queryQ.createTailer(), 1))
+        {
+            readQueues = arguments.stream().map(s -> ChronicleQueueBuilder.single(s).readOnly(true).build()).collect(Collectors.toList());
+            List<Iterator<ResultHandler.ComparableResultSet>> its = readQueues.stream().map(q -> new StoredResultSetIterator(q.createTailer())).collect(Collectors.toList());
+            while (queries.hasNext())
+                rh.handleResults(queries.next(), resultSets(its));
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+        finally
+        {
+            if (readQueues != null)
+                readQueues.forEach(Closeable::close);
+        }
+    }
+
+    @VisibleForTesting
+    public static List<ResultHandler.ComparableResultSet> resultSets(List<Iterator<ResultHandler.ComparableResultSet>> its)
+    {
+        List<ResultHandler.ComparableResultSet> resultSets = new ArrayList<>(its.size());
+        for (Iterator<ResultHandler.ComparableResultSet> it : its)
+        {
+            if (it.hasNext())
+                resultSets.add(it.next());
+            else
+                resultSets.add(null);
+        }
+        return resultSets;
+    }
+
+    @VisibleForTesting
+    public static class StoredResultSetIterator extends AbstractIterator<ResultHandler.ComparableResultSet>
+    {
+        private final ExcerptTailer tailer;
+
+        public StoredResultSetIterator(ExcerptTailer tailer)
+        {
+            this.tailer = tailer;
+        }
+
+        protected ResultHandler.ComparableResultSet computeNext()
+        {
+            StoredResultSet srs = StoredResultSet.fromTailer(tailer);
+            if (srs.hasMoreResultSets)
+                return srs;
+            return endOfData();
+        }
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Dump.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Dump.java b/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Dump.java
new file mode 100644
index 0000000..1263a11
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Dump.java
@@ -0,0 +1,325 @@
+/*
+ * 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.fqltool.commands;
+
+import java.io.File;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import io.airlift.airline.Option;
+import io.netty.buffer.Unpooled;
+import net.openhft.chronicle.bytes.Bytes;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+import net.openhft.chronicle.queue.ExcerptTailer;
+import net.openhft.chronicle.queue.RollCycles;
+import net.openhft.chronicle.threads.Pauser;
+import net.openhft.chronicle.wire.ReadMarshallable;
+import net.openhft.chronicle.wire.ValueIn;
+import net.openhft.chronicle.wire.WireIn;
+import org.apache.cassandra.audit.FullQueryLogger;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * Dump the contents of a list of paths containing full query logs
+ */
+@Command(name = "dump", description = "Dump the contents of a full query log")
+public class Dump implements Runnable
+{
+    static final char[] HEXI_DECIMAL = "0123456789ABCDEF".toCharArray();
+
+    @Arguments(usage = "<path1> [<path2>...<pathN>]", description = "Path containing the full query logs to dump.", required = true)
+    private List<String> arguments = new ArrayList<>();
+
+    @Option(title = "roll_cycle", name = {"--roll-cycle"}, description = "How often to roll the log file was rolled. May be necessary for Chronicle to correctly parse file names. (MINUTELY, HOURLY, DAILY). Default HOURLY.")
+    private String rollCycle = "HOURLY";
+
+    @Option(title = "follow", name = {"--follow"}, description = "Upon reacahing the end of the log continue indefinitely waiting for more records")
+    private boolean follow = false;
+
+    @Override
+    public void run()
+    {
+        dump(arguments, rollCycle, follow);
+    }
+
+    public static void dump(List<String> arguments, String rollCycle, boolean follow)
+    {
+        StringBuilder sb = new StringBuilder();
+        ReadMarshallable reader = wireIn ->
+        {
+            sb.setLength(0);
+
+            int version = wireIn.read(FullQueryLogger.VERSION).int16();
+            if (version != FullQueryLogger.CURRENT_VERSION)
+                throw new UnsupportedOperationException("Full query log of unexpected version " + version + " encountered");
+
+            String type = wireIn.read(FullQueryLogger.TYPE).text();
+            sb.append("Type: ")
+              .append(type)
+              .append(System.lineSeparator());
+
+            long queryStartTime = wireIn.read(FullQueryLogger.QUERY_START_TIME).int64();
+            sb.append("Query start time: ")
+              .append(queryStartTime)
+              .append(System.lineSeparator());
+
+            int protocolVersion = wireIn.read(FullQueryLogger.PROTOCOL_VERSION).int32();
+            sb.append("Protocol version: ")
+              .append(protocolVersion)
+              .append(System.lineSeparator());
+
+            QueryOptions options =
+                QueryOptions.codec.decode(Unpooled.wrappedBuffer(wireIn.read(FullQueryLogger.QUERY_OPTIONS).bytes()),
+                                          ProtocolVersion.decode(protocolVersion));
+
+            long generatedTimestamp = wireIn.read(FullQueryLogger.GENERATED_TIMESTAMP).int64();
+            sb.append("Generated timestamp:")
+              .append(generatedTimestamp)
+              .append(System.lineSeparator());
+
+            int generatedNowInSeconds = wireIn.read(FullQueryLogger.GENERATED_NOW_IN_SECONDS).int32();
+            sb.append("Generated nowInSeconds:")
+              .append(generatedNowInSeconds)
+              .append(System.lineSeparator());
+
+            switch (type)
+            {
+                case (FullQueryLogger.SINGLE_QUERY):
+                    dumpQuery(options, wireIn, sb);
+                    break;
+
+                case (FullQueryLogger.BATCH):
+                    dumpBatch(options, wireIn, sb);
+                    break;
+
+                default:
+                    throw new UnsupportedOperationException("Log entry of unsupported type " + type);
+            }
+
+            System.out.print(sb.toString());
+            System.out.flush();
+        };
+
+        //Backoff strategy for spinning on the queue, not aggressive at all as this doesn't need to be low latency
+        Pauser pauser = Pauser.millis(100);
+        List<ChronicleQueue> queues = arguments.stream().distinct().map(path -> ChronicleQueueBuilder.single(new File(path)).readOnly(true).rollCycle(RollCycles.valueOf(rollCycle)).build()).collect(Collectors.toList());
+        List<ExcerptTailer> tailers = queues.stream().map(ChronicleQueue::createTailer).collect(Collectors.toList());
+        boolean hadWork = true;
+        while (hadWork)
+        {
+            hadWork = false;
+            for (ExcerptTailer tailer : tailers)
+            {
+                while (tailer.readDocument(reader))
+                {
+                    hadWork = true;
+                }
+            }
+
+            if (follow)
+            {
+                if (!hadWork)
+                {
+                    //Chronicle queue doesn't support blocking so use this backoff strategy
+                    pauser.pause();
+                }
+                //Don't terminate the loop even if there wasn't work
+                hadWork = true;
+            }
+        }
+    }
+
+    private static void dumpQuery(QueryOptions options, WireIn wireIn, StringBuilder sb)
+    {
+        sb.append("Query: ")
+          .append(wireIn.read(FullQueryLogger.QUERY).text())
+          .append(System.lineSeparator());
+
+        List<ByteBuffer> values = options.getValues() != null
+                                ? options.getValues()
+                                : Collections.emptyList();
+
+        sb.append("Values: ")
+          .append(System.lineSeparator());
+        appendValuesToStringBuilder(values, sb);
+        sb.append(System.lineSeparator());
+    }
+
+    private static void dumpBatch(QueryOptions options, WireIn wireIn, StringBuilder sb)
+    {
+        sb.append("Batch type: ")
+          .append(wireIn.read(FullQueryLogger.BATCH_TYPE).text())
+          .append(System.lineSeparator());
+
+        ValueIn in = wireIn.read(FullQueryLogger.QUERIES);
+        int numQueries = in.int32();
+        List<String> queries = new ArrayList<>(numQueries);
+        for (int i = 0; i < numQueries; i++)
+            queries.add(in.text());
+
+        in = wireIn.read(FullQueryLogger.VALUES);
+        int numValues = in.int32();
+
+        for (int i = 0; i < numValues; i++)
+        {
+            int numSubValues = in.int32();
+            List<ByteBuffer> subValues = new ArrayList<>(numSubValues);
+            for (int j = 0; j < numSubValues; j++)
+                subValues.add(ByteBuffer.wrap(in.bytes()));
+
+            sb.append("Query: ")
+              .append(queries.get(i))
+              .append(System.lineSeparator());
+
+            sb.append("Values: ")
+              .append(System.lineSeparator());
+            appendValuesToStringBuilder(subValues, sb);
+        }
+
+        sb.append(System.lineSeparator());
+    }
+
+    private static void appendValuesToStringBuilder(List<ByteBuffer> values, StringBuilder sb)
+    {
+        boolean first = true;
+        for (ByteBuffer value : values)
+        {
+            Bytes bytes = Bytes.wrapForRead(value);
+            long maxLength2 = Math.min(1024, bytes.readLimit() - bytes.readPosition());
+            toHexString(bytes, bytes.readPosition(), maxLength2, sb);
+            if (maxLength2 < bytes.readLimit() - bytes.readPosition())
+            {
+                sb.append("... truncated").append(System.lineSeparator());
+            }
+
+            if (first)
+            {
+                first = false;
+            }
+            else
+            {
+                sb.append("-----").append(System.lineSeparator());
+            }
+        }
+    }
+
+    //This is from net.openhft.chronicle.bytes, need to pass in the StringBuilder so had to copy
+    /*
+     * Copyright 2016 higherfrequencytrading.com
+     *
+     * Licensed 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.
+     */
+    /**
+     * display the hex data of {@link Bytes} from the position() to the limit()
+     *
+     * @param bytes the buffer you wish to toString()
+     * @return hex representation of the buffer, from example [0D ,OA, FF]
+     */
+    public static String toHexString(final Bytes bytes, long offset, long len, StringBuilder builder)
+    throws BufferUnderflowException
+    {
+        if (len == 0)
+            return "";
+
+        int width = 16;
+        int[] lastLine = new int[width];
+        String sep = "";
+        long position = bytes.readPosition();
+        long limit = bytes.readLimit();
+
+        try {
+            bytes.readPositionRemaining(offset, len);
+
+            long start = offset / width * width;
+            long end = (offset + len + width - 1) / width * width;
+            for (long i = start; i < end; i += width) {
+                // check for duplicate rows
+                if (i + width < end) {
+                    boolean same = true;
+
+                    for (int j = 0; j < width && i + j < offset + len; j++) {
+                        int ch = bytes.readUnsignedByte(i + j);
+                        same &= (ch == lastLine[j]);
+                        lastLine[j] = ch;
+                    }
+                    if (i > start && same) {
+                        sep = "........\n";
+                        continue;
+                    }
+                }
+                builder.append(sep);
+                sep = "";
+                String str = Long.toHexString(i);
+                for (int j = str.length(); j < 8; j++)
+                    builder.append('0');
+                builder.append(str);
+                for (int j = 0; j < width; j++) {
+                    if (j == width / 2)
+                        builder.append(' ');
+                    if (i + j < offset || i + j >= offset + len) {
+                        builder.append("   ");
+
+                    } else {
+                        builder.append(' ');
+                        int ch = bytes.readUnsignedByte(i + j);
+                        builder.append(HEXI_DECIMAL[ch >> 4]);
+                        builder.append(HEXI_DECIMAL[ch & 15]);
+                    }
+                }
+                builder.append(' ');
+                for (int j = 0; j < width; j++) {
+                    if (j == width / 2)
+                        builder.append(' ');
+                    if (i + j < offset || i + j >= offset + len) {
+                        builder.append(' ');
+
+                    } else {
+                        int ch = bytes.readUnsignedByte(i + j);
+                        if (ch < ' ' || ch > 126)
+                            ch = '\u00B7';
+                        builder.append((char) ch);
+                    }
+                }
+                builder.append("\n");
+            }
+            return builder.toString();
+        } finally {
+            bytes.readLimit(limit);
+            bytes.readPosition(position);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Replay.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Replay.java b/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Replay.java
new file mode 100644
index 0000000..adea742
--- /dev/null
+++ b/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Replay.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.fqltool.commands;
+
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import io.airlift.airline.Option;
+import net.openhft.chronicle.core.io.Closeable;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+
+import org.apache.cassandra.fqltool.FQLQuery;
+import org.apache.cassandra.fqltool.FQLQueryIterator;
+import org.apache.cassandra.fqltool.QueryReplayer;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.MergeIterator;
+
+/**
+ * replay the contents of a list of paths containing full query logs
+ */
+@Command(name = "replay", description = "Replay full query logs")
+public class Replay implements Runnable
+{
+    @Arguments(usage = "<path1> [<path2>...<pathN>]", description = "Paths containing the full query logs to replay.", required = true)
+    private List<String> arguments = new ArrayList<>();
+
+    @Option(title = "target", name = {"--target"}, description = "Hosts to replay the logs to, can be repeated to replay to more hosts.")
+    private List<String> targetHosts;
+
+    @Option(title = "results", name = { "--results"}, description = "Where to store the results of the queries, this should be a directory. Leave this option out to avoid storing results.")
+    private String resultPath;
+
+    @Option(title = "keyspace", name = { "--keyspace"}, description = "Only replay queries against this keyspace and queries without keyspace set.")
+    private String keyspace;
+
+    @Option(title = "debug", name = {"--debug"}, description = "Debug mode, print all queries executed.")
+    private boolean debug;
+
+    @Option(title = "store_queries", name = {"--store-queries"}, description = "Path to store the queries executed. Stores queries in the same order as the result sets are in the result files. Requires --results")
+    private String queryStorePath;
+
+    @Override
+    public void run()
+    {
+        try
+        {
+            List<File> resultPaths = null;
+            if (resultPath != null)
+            {
+                File basePath = new File(resultPath);
+                if (!basePath.exists() || !basePath.isDirectory())
+                {
+                    System.err.println("The results path (" + basePath + ") should be an existing directory");
+                    System.exit(1);
+                }
+                resultPaths = targetHosts.stream().map(target -> new File(basePath, target)).collect(Collectors.toList());
+                resultPaths.forEach(File::mkdir);
+            }
+            if (targetHosts.size() < 1)
+            {
+                System.err.println("You need to state at least one --target host to replay the query against");
+                System.exit(1);
+            }
+            replay(keyspace, arguments, targetHosts, resultPaths, queryStorePath, debug);
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static void replay(String keyspace, List<String> arguments, List<String> targetHosts, List<File> resultPaths, String queryStorePath, boolean debug)
+    {
+        int readAhead = 200; // how many fql queries should we read in to memory to be able to sort them?
+        List<ChronicleQueue> readQueues = null;
+        List<FQLQueryIterator> iterators = null;
+        List<Predicate<FQLQuery>> filters = new ArrayList<>();
+
+        if (keyspace != null)
+            filters.add(fqlQuery -> fqlQuery.keyspace() == null || fqlQuery.keyspace().equals(keyspace));
+
+        try
+        {
+            readQueues = arguments.stream().map(s -> ChronicleQueueBuilder.single(s).readOnly(true).build()).collect(Collectors.toList());
+            iterators = readQueues.stream().map(ChronicleQueue::createTailer).map(tailer -> new FQLQueryIterator(tailer, readAhead)).collect(Collectors.toList());
+            try (MergeIterator<FQLQuery, List<FQLQuery>> iter = MergeIterator.get(iterators, FQLQuery::compareTo, new Reducer());
+                 QueryReplayer replayer = new QueryReplayer(iter, targetHosts, resultPaths, filters, System.out, queryStorePath, debug))
+            {
+                replayer.replay();
+            }
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+        finally
+        {
+            if (iterators != null)
+                iterators.forEach(AbstractIterator::close);
+            if (readQueues != null)
+                readQueues.forEach(Closeable::close);
+        }
+    }
+
+    @VisibleForTesting
+    public static class Reducer extends MergeIterator.Reducer<FQLQuery, List<FQLQuery>>
+    {
+        List<FQLQuery> queries = new ArrayList<>();
+        public void reduce(int idx, FQLQuery current)
+        {
+            queries.add(current);
+        }
+
+        protected List<FQLQuery> getReduced()
+        {
+            return queries;
+        }
+        protected void onKeyChange()
+        {
+            queries.clear();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f83bd5ac/tools/fqltool/test/unit/org/apache/cassandra/fqltool/FQLCompareTest.java
----------------------------------------------------------------------
diff --git a/tools/fqltool/test/unit/org/apache/cassandra/fqltool/FQLCompareTest.java b/tools/fqltool/test/unit/org/apache/cassandra/fqltool/FQLCompareTest.java
new file mode 100644
index 0000000..7990b7e
--- /dev/null
+++ b/tools/fqltool/test/unit/org/apache/cassandra/fqltool/FQLCompareTest.java
@@ -0,0 +1,131 @@
+/*
+ * 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.fqltool;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import net.openhft.chronicle.core.io.Closeable;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.fqltool.commands.Compare;
+import org.apache.cassandra.tools.Util;
+
+
+import static org.psjava.util.AssertStatus.assertTrue;
+
+public class FQLCompareTest
+{
+    public FQLCompareTest()
+    {
+        Util.initDatabaseDescriptor();
+    }
+
+    @Test
+    public void endToEnd() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = generateResultSets(targetHosts, tmpDir, queryDir, true, false);
+        Compare.compare(queryDir.toString(), resultPaths.stream().map(File::toString).collect(Collectors.toList()));
+    }
+
+    @Test
+    public void endToEndQueryFailures() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = generateResultSets(targetHosts, tmpDir, queryDir, true,true);
+        Compare.compare(queryDir.toString(), resultPaths.stream().map(File::toString).collect(Collectors.toList()));
+    }
+
+    @Test
+    public void compareEqual() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = generateResultSets(targetHosts, tmpDir, queryDir, false,false);
+
+        ResultComparator comparator = new ResultComparator();
+        List<ChronicleQueue> readQueues = null;
+        try
+        {
+            readQueues = resultPaths.stream().map(s -> ChronicleQueueBuilder.single(s).readOnly(true).build()).collect(Collectors.toList());
+            List<Iterator<ResultHandler.ComparableResultSet>> its = readQueues.stream().map(q -> new Compare.StoredResultSetIterator(q.createTailer())).collect(Collectors.toList());
+            List<ResultHandler.ComparableResultSet> resultSets = Compare.resultSets(its);
+            while(resultSets.stream().allMatch(Objects::nonNull))
+            {
+                assertTrue(comparator.compareColumnDefinitions(targetHosts, query(), resultSets.stream().map(ResultHandler.ComparableResultSet::getColumnDefinitions).collect(Collectors.toList())));
+                List<Iterator<ResultHandler.ComparableRow>> rows = resultSets.stream().map(Iterable::iterator).collect(Collectors.toList());
+
+                List<ResultHandler.ComparableRow> toCompare = ResultHandler.rows(rows);
+
+                while (toCompare.stream().allMatch(Objects::nonNull))
+                {
+                    assertTrue(comparator.compareRows(targetHosts, query(), ResultHandler.rows(rows)));
+                    toCompare = ResultHandler.rows(rows);
+                }
+                resultSets = Compare.resultSets(its);
+            }
+        }
+        finally
+        {
+            if (readQueues != null)
+                readQueues.forEach(Closeable::close);
+        }
+    }
+
+    private List<File> generateResultSets(List<String> targetHosts, File resultDir, File queryDir, boolean random, boolean includeFailures) throws IOException
+    {
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(resultDir, host); f.mkdir(); resultPaths.add(f);});
+
+        try (ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir))
+        {
+            for (int i = 0; i < 100; i++)
+            {
+                ResultHandler.ComparableResultSet resultSet1 = includeFailures && (i % 10 == 0)
+                                                               ? StoredResultSet.failed("test failure!")
+                                                               : FQLReplayTest.createResultSet(10, 10, random);
+                ResultHandler.ComparableResultSet resultSet2 = FQLReplayTest.createResultSet(10, 10, random);
+                rh.handleResults(query(), Lists.newArrayList(resultSet1, resultSet2));
+            }
+        }
+        return resultPaths;
+    }
+
+    private FQLQuery.Single query()
+    {
+        return new FQLQuery.Single("abc", QueryOptions.DEFAULT.getProtocolVersion().asInt(), QueryOptions.DEFAULT, 12345, 5555, 6666, "select * from xyz", Collections.emptyList());
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[3/7] cassandra git commit: Add fqltool replay

Posted by ma...@apache.org.
Add fqltool replay

Patch by marcuse; reviewed by Jason Brown and Dinesh Joshi for CASSANDRA-14618


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

Branch: refs/heads/trunk
Commit: 62ffb7723917768c38c9e012710c6dce509191c1
Parents: 46c33f3
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Aug 6 16:32:27 2018 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Sat Sep 1 08:35:54 2018 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/audit/FullQueryLogger.java |   5 +-
 .../apache/cassandra/service/QueryState.java    |   8 +
 .../cassandra/tools/FullQueryLogTool.java       |   6 +-
 .../tools/fqltool/DriverResultSet.java          | 241 ++++++
 .../apache/cassandra/tools/fqltool/Dump.java    | 325 --------
 .../cassandra/tools/fqltool/FQLQuery.java       | 278 +++++++
 .../tools/fqltool/FQLQueryIterator.java         |  72 ++
 .../cassandra/tools/fqltool/FQLQueryReader.java | 116 +++
 .../cassandra/tools/fqltool/QueryReplayer.java  | 167 ++++
 .../tools/fqltool/ResultComparator.java         | 116 +++
 .../cassandra/tools/fqltool/ResultHandler.java  | 124 +++
 .../cassandra/tools/fqltool/ResultStore.java    | 142 ++++
 .../cassandra/tools/fqltool/commands/Dump.java  | 325 ++++++++
 .../tools/fqltool/commands/Replay.java          | 148 ++++
 .../cassandra/tools/fqltool/FQLReplayTest.java  | 760 +++++++++++++++++++
 16 files changed, 2505 insertions(+), 329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index cd2a14a..1227337 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Add fqltool replay (CASSANDRA-14618)
  * Log keyspace in full query log (CASSANDRA-14656)
  * Transient Replication and Cheap Quorums (CASSANDRA-14404)
  * Log server-generated timestamp and nowInSeconds used by queries in FQL (CASSANDRA-14675)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/audit/FullQueryLogger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/audit/FullQueryLogger.java b/src/java/org/apache/cassandra/audit/FullQueryLogger.java
index c9f8447..9c1f472 100644
--- a/src/java/org/apache/cassandra/audit/FullQueryLogger.java
+++ b/src/java/org/apache/cassandra/audit/FullQueryLogger.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import javax.annotation.Nullable;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Ints;
 
@@ -151,7 +152,7 @@ public class FullQueryLogger extends BinLogAuditLogger implements IAuditLogger
         logRecord(wrappedQuery, binLog);
     }
 
-    static class Query extends AbstractLogEntry
+    public static class Query extends AbstractLogEntry
     {
         private final String query;
 
@@ -181,7 +182,7 @@ public class FullQueryLogger extends BinLogAuditLogger implements IAuditLogger
         }
     }
 
-    static class Batch extends AbstractLogEntry
+    public static class Batch extends AbstractLogEntry
     {
         private final int weight;
         private final BatchStatement.Type batchType;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/service/QueryState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/QueryState.java b/src/java/org/apache/cassandra/service/QueryState.java
index 2bd07ab..26f58bf 100644
--- a/src/java/org/apache/cassandra/service/QueryState.java
+++ b/src/java/org/apache/cassandra/service/QueryState.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.service;
 
 import java.net.InetAddress;
 
+import org.apache.cassandra.transport.ClientStat;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -39,6 +40,13 @@ public class QueryState
         this.clientState = clientState;
     }
 
+    public QueryState(ClientState clientState, long timestamp, int nowInSeconds)
+    {
+        this(clientState);
+        this.timestamp = timestamp;
+        this.nowInSeconds = nowInSeconds;
+    }
+
     /**
      * @return a QueryState object for internal C* calls (not limited by any kind of auth).
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/FullQueryLogTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/FullQueryLogTool.java b/src/java/org/apache/cassandra/tools/FullQueryLogTool.java
index 0d170d9..c1d4713 100644
--- a/src/java/org/apache/cassandra/tools/FullQueryLogTool.java
+++ b/src/java/org/apache/cassandra/tools/FullQueryLogTool.java
@@ -31,7 +31,8 @@ import io.airlift.airline.ParseCommandUnrecognizedException;
 import io.airlift.airline.ParseOptionConversionException;
 import io.airlift.airline.ParseOptionMissingException;
 import io.airlift.airline.ParseOptionMissingValueException;
-import org.apache.cassandra.tools.fqltool.Dump;
+import org.apache.cassandra.tools.fqltool.commands.Dump;
+import org.apache.cassandra.tools.fqltool.commands.Replay;
 
 import static com.google.common.base.Throwables.getStackTraceAsString;
 import static com.google.common.collect.Lists.newArrayList;
@@ -42,7 +43,8 @@ public class FullQueryLogTool
     {
         List<Class<? extends Runnable>> commands = newArrayList(
                 Help.class,
-                Dump.class
+                Dump.class,
+                Replay.class
         );
 
         Cli.CliBuilder<Runnable> builder = Cli.builder("fqltool");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/DriverResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/DriverResultSet.java b/src/java/org/apache/cassandra/tools/fqltool/DriverResultSet.java
new file mode 100644
index 0000000..6c4ee45
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/DriverResultSet.java
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.AbstractIterator;
+
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+/**
+ * Wraps a result set from the driver so that we can reuse the compare code when reading
+ * up a result set produced by ResultStore.
+ */
+public class DriverResultSet implements ResultHandler.ComparableResultSet
+{
+    private final ResultSet resultSet;
+    private final Throwable failureException;
+
+    public DriverResultSet(ResultSet resultSet)
+    {
+        this(resultSet, null);
+    }
+
+    private DriverResultSet(ResultSet res, Throwable failureException)
+    {
+        resultSet = res;
+        this.failureException = failureException;
+    }
+
+    public static DriverResultSet failed(Throwable ex)
+    {
+        return new DriverResultSet(null, ex);
+    }
+
+    public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+    {
+        if (wasFailed())
+            return new DriverColumnDefinitions(null, true);
+
+        return new DriverColumnDefinitions(resultSet.getColumnDefinitions());
+    }
+
+    public boolean wasFailed()
+    {
+        return failureException != null;
+    }
+
+    public Throwable getFailureException()
+    {
+        return failureException;
+    }
+
+    public Iterator<ResultHandler.ComparableRow> iterator()
+    {
+        if (wasFailed())
+            return Collections.emptyListIterator();
+        return new AbstractIterator<ResultHandler.ComparableRow>()
+        {
+            Iterator<Row> iter = resultSet.iterator();
+            protected ResultHandler.ComparableRow computeNext()
+            {
+                if (iter.hasNext())
+                    return new DriverRow(iter.next());
+                return endOfData();
+            }
+        };
+    }
+
+    public static class DriverRow implements ResultHandler.ComparableRow
+    {
+        private final Row row;
+
+        public DriverRow(Row row)
+        {
+            this.row = row;
+        }
+
+        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+        {
+            return new DriverColumnDefinitions(row.getColumnDefinitions());
+        }
+
+        public ByteBuffer getBytesUnsafe(int i)
+        {
+            return row.getBytesUnsafe(i);
+        }
+
+        @Override
+        public boolean equals(Object oo)
+        {
+            if (!(oo instanceof ResultHandler.ComparableRow))
+                return false;
+
+            ResultHandler.ComparableRow o = (ResultHandler.ComparableRow)oo;
+            if (getColumnDefinitions().size() != o.getColumnDefinitions().size())
+                return false;
+
+            for (int j = 0; j < getColumnDefinitions().size(); j++)
+            {
+                ByteBuffer b1 = getBytesUnsafe(j);
+                ByteBuffer b2 = o.getBytesUnsafe(j);
+
+                if (b1 != null && b2 != null && !b1.equals(b2))
+                {
+                    return false;
+                }
+                if (b1 == null && b2 != null || b2 == null && b1 != null)
+                {
+                    return false;
+                }
+            }
+            return true;
+        }
+
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder();
+            List<ResultHandler.ComparableDefinition> colDefs = getColumnDefinitions().asList();
+            for (int i = 0; i < getColumnDefinitions().size(); i++)
+            {
+                ByteBuffer bb = getBytesUnsafe(i);
+                String row = bb != null ? ByteBufferUtil.bytesToHex(bb) : "NULL";
+                sb.append(colDefs.get(i)).append(':').append(row).append(",");
+            }
+            return sb.toString();
+        }
+    }
+
+    public static class DriverColumnDefinitions implements ResultHandler.ComparableColumnDefinitions
+    {
+        private final ColumnDefinitions columnDefinitions;
+        private final boolean failed;
+
+        public DriverColumnDefinitions(ColumnDefinitions columnDefinitions)
+        {
+            this(columnDefinitions, false);
+        }
+
+        private DriverColumnDefinitions(ColumnDefinitions columnDefinitions, boolean failed)
+        {
+            this.columnDefinitions = columnDefinitions;
+            this.failed = failed;
+        }
+
+        public List<ResultHandler.ComparableDefinition> asList()
+        {
+            if (wasFailed())
+                return Collections.emptyList();
+            return columnDefinitions.asList().stream().map(DriverDefinition::new).collect(Collectors.toList());
+        }
+
+        public boolean wasFailed()
+        {
+            return failed;
+        }
+
+        public int size()
+        {
+            return columnDefinitions.size();
+        }
+
+        public Iterator<ResultHandler.ComparableDefinition> iterator()
+        {
+            return asList().iterator();
+        }
+
+        public boolean equals(Object oo)
+        {
+            if (!(oo instanceof ResultHandler.ComparableColumnDefinitions))
+                return false;
+
+            ResultHandler.ComparableColumnDefinitions o = (ResultHandler.ComparableColumnDefinitions)oo;
+            if (wasFailed() && o.wasFailed())
+                return true;
+
+            if (size() != o.size())
+                return false;
+
+            return asList().equals(o.asList());
+        }
+    }
+
+    public static class DriverDefinition implements ResultHandler.ComparableDefinition
+    {
+        private final ColumnDefinitions.Definition def;
+
+        public DriverDefinition(ColumnDefinitions.Definition def)
+        {
+            this.def = def;
+        }
+
+        public String getType()
+        {
+            return def.getType().toString();
+        }
+
+        public String getName()
+        {
+            return def.getName();
+        }
+
+        public boolean equals(Object oo)
+        {
+            if (!(oo instanceof ResultHandler.ComparableDefinition))
+                return false;
+
+            return def.equals(((DriverDefinition)oo).def);
+        }
+
+        public String toString()
+        {
+            return getName() + ':' + getType();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/Dump.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/Dump.java b/src/java/org/apache/cassandra/tools/fqltool/Dump.java
deleted file mode 100644
index a8e7592..0000000
--- a/src/java/org/apache/cassandra/tools/fqltool/Dump.java
+++ /dev/null
@@ -1,325 +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.tools.fqltool;
-
-import java.io.File;
-import java.nio.BufferUnderflowException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import io.airlift.airline.Arguments;
-import io.airlift.airline.Command;
-import io.airlift.airline.Option;
-import io.netty.buffer.Unpooled;
-import net.openhft.chronicle.bytes.Bytes;
-import net.openhft.chronicle.queue.ChronicleQueue;
-import net.openhft.chronicle.queue.ChronicleQueueBuilder;
-import net.openhft.chronicle.queue.ExcerptTailer;
-import net.openhft.chronicle.queue.RollCycles;
-import net.openhft.chronicle.threads.Pauser;
-import net.openhft.chronicle.wire.ReadMarshallable;
-import net.openhft.chronicle.wire.ValueIn;
-import net.openhft.chronicle.wire.WireIn;
-import org.apache.cassandra.audit.FullQueryLogger;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.transport.ProtocolVersion;
-
-/**
- * Dump the contents of a list of paths containing full query logs
- */
-@Command(name = "dump", description = "Dump the contents of a full query log")
-public class Dump implements Runnable
-{
-    static final char[] HEXI_DECIMAL = "0123456789ABCDEF".toCharArray();
-
-    @Arguments(usage = "<path1> [<path2>...<pathN>]", description = "Path containing the full query logs to dump.", required = true)
-    private List<String> arguments = new ArrayList<>();
-
-    @Option(title = "roll_cycle", name = {"--roll-cycle"}, description = "How often to roll the log file was rolled. May be necessary for Chronicle to correctly parse file names. (MINUTELY, HOURLY, DAILY). Default HOURLY.")
-    private String rollCycle = "HOURLY";
-
-    @Option(title = "follow", name = {"--follow"}, description = "Upon reacahing the end of the log continue indefinitely waiting for more records")
-    private boolean follow = false;
-
-    @Override
-    public void run()
-    {
-        dump(arguments, rollCycle, follow);
-    }
-
-    public static void dump(List<String> arguments, String rollCycle, boolean follow)
-    {
-        StringBuilder sb = new StringBuilder();
-        ReadMarshallable reader = wireIn ->
-        {
-            sb.setLength(0);
-
-            int version = wireIn.read(FullQueryLogger.VERSION).int16();
-            if (version != FullQueryLogger.CURRENT_VERSION)
-                throw new UnsupportedOperationException("Full query log of unexpected version " + version + " encountered");
-
-            String type = wireIn.read(FullQueryLogger.TYPE).text();
-            sb.append("Type: ")
-              .append(type)
-              .append(System.lineSeparator());
-
-            long queryStartTime = wireIn.read(FullQueryLogger.QUERY_START_TIME).int64();
-            sb.append("Query start time: ")
-              .append(queryStartTime)
-              .append(System.lineSeparator());
-
-            int protocolVersion = wireIn.read(FullQueryLogger.PROTOCOL_VERSION).int32();
-            sb.append("Protocol version: ")
-              .append(protocolVersion)
-              .append(System.lineSeparator());
-
-            QueryOptions options =
-                QueryOptions.codec.decode(Unpooled.wrappedBuffer(wireIn.read(FullQueryLogger.QUERY_OPTIONS).bytes()),
-                                          ProtocolVersion.decode(protocolVersion));
-
-            long generatedTimestamp = wireIn.read(FullQueryLogger.GENERATED_TIMESTAMP).int64();
-            sb.append("Generated timestamp:")
-              .append(generatedTimestamp)
-              .append(System.lineSeparator());
-
-            int generatedNowInSeconds = wireIn.read(FullQueryLogger.GENERATED_NOW_IN_SECONDS).int32();
-            sb.append("Generated nowInSeconds:")
-              .append(generatedNowInSeconds)
-              .append(System.lineSeparator());
-
-            switch (type)
-            {
-                case (FullQueryLogger.SINGLE_QUERY):
-                    dumpQuery(options, wireIn, sb);
-                    break;
-
-                case (FullQueryLogger.BATCH):
-                    dumpBatch(options, wireIn, sb);
-                    break;
-
-                default:
-                    throw new UnsupportedOperationException("Log entry of unsupported type " + type);
-            }
-
-            System.out.print(sb.toString());
-            System.out.flush();
-        };
-
-        //Backoff strategy for spinning on the queue, not aggressive at all as this doesn't need to be low latency
-        Pauser pauser = Pauser.millis(100);
-        List<ChronicleQueue> queues = arguments.stream().distinct().map(path -> ChronicleQueueBuilder.single(new File(path)).readOnly(true).rollCycle(RollCycles.valueOf(rollCycle)).build()).collect(Collectors.toList());
-        List<ExcerptTailer> tailers = queues.stream().map(ChronicleQueue::createTailer).collect(Collectors.toList());
-        boolean hadWork = true;
-        while (hadWork)
-        {
-            hadWork = false;
-            for (ExcerptTailer tailer : tailers)
-            {
-                while (tailer.readDocument(reader))
-                {
-                    hadWork = true;
-                }
-            }
-
-            if (follow)
-            {
-                if (!hadWork)
-                {
-                    //Chronicle queue doesn't support blocking so use this backoff strategy
-                    pauser.pause();
-                }
-                //Don't terminate the loop even if there wasn't work
-                hadWork = true;
-            }
-        }
-    }
-
-    private static void dumpQuery(QueryOptions options, WireIn wireIn, StringBuilder sb)
-    {
-        sb.append("Query: ")
-          .append(wireIn.read(FullQueryLogger.QUERY).text())
-          .append(System.lineSeparator());
-
-        List<ByteBuffer> values = options.getValues() != null
-                                ? options.getValues()
-                                : Collections.emptyList();
-
-        sb.append("Values: ")
-          .append(System.lineSeparator());
-        appendValuesToStringBuilder(values, sb);
-        sb.append(System.lineSeparator());
-    }
-
-    private static void dumpBatch(QueryOptions options, WireIn wireIn, StringBuilder sb)
-    {
-        sb.append("Batch type: ")
-          .append(wireIn.read(FullQueryLogger.BATCH_TYPE).text())
-          .append(System.lineSeparator());
-
-        ValueIn in = wireIn.read(FullQueryLogger.QUERIES);
-        int numQueries = in.int32();
-        List<String> queries = new ArrayList<>(numQueries);
-        for (int i = 0; i < numQueries; i++)
-            queries.add(in.text());
-
-        in = wireIn.read(FullQueryLogger.VALUES);
-        int numValues = in.int32();
-
-        for (int i = 0; i < numValues; i++)
-        {
-            int numSubValues = in.int32();
-            List<ByteBuffer> subValues = new ArrayList<>(numSubValues);
-            for (int j = 0; j < numSubValues; j++)
-                subValues.add(ByteBuffer.wrap(in.bytes()));
-
-            sb.append("Query: ")
-              .append(queries.get(i))
-              .append(System.lineSeparator());
-
-            sb.append("Values: ")
-              .append(System.lineSeparator());
-            appendValuesToStringBuilder(subValues, sb);
-        }
-
-        sb.append(System.lineSeparator());
-    }
-
-    private static void appendValuesToStringBuilder(List<ByteBuffer> values, StringBuilder sb)
-    {
-        boolean first = true;
-        for (ByteBuffer value : values)
-        {
-            Bytes bytes = Bytes.wrapForRead(value);
-            long maxLength2 = Math.min(1024, bytes.readLimit() - bytes.readPosition());
-            toHexString(bytes, bytes.readPosition(), maxLength2, sb);
-            if (maxLength2 < bytes.readLimit() - bytes.readPosition())
-            {
-                sb.append("... truncated").append(System.lineSeparator());
-            }
-
-            if (first)
-            {
-                first = false;
-            }
-            else
-            {
-                sb.append("-----").append(System.lineSeparator());
-            }
-        }
-    }
-
-    //This is from net.openhft.chronicle.bytes, need to pass in the StringBuilder so had to copy
-    /*
-     * Copyright 2016 higherfrequencytrading.com
-     *
-     * Licensed 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.
-     */
-    /**
-     * display the hex data of {@link Bytes} from the position() to the limit()
-     *
-     * @param bytes the buffer you wish to toString()
-     * @return hex representation of the buffer, from example [0D ,OA, FF]
-     */
-    public static String toHexString(final Bytes bytes, long offset, long len, StringBuilder builder)
-    throws BufferUnderflowException
-    {
-        if (len == 0)
-            return "";
-
-        int width = 16;
-        int[] lastLine = new int[width];
-        String sep = "";
-        long position = bytes.readPosition();
-        long limit = bytes.readLimit();
-
-        try {
-            bytes.readPositionRemaining(offset, len);
-
-            long start = offset / width * width;
-            long end = (offset + len + width - 1) / width * width;
-            for (long i = start; i < end; i += width) {
-                // check for duplicate rows
-                if (i + width < end) {
-                    boolean same = true;
-
-                    for (int j = 0; j < width && i + j < offset + len; j++) {
-                        int ch = bytes.readUnsignedByte(i + j);
-                        same &= (ch == lastLine[j]);
-                        lastLine[j] = ch;
-                    }
-                    if (i > start && same) {
-                        sep = "........\n";
-                        continue;
-                    }
-                }
-                builder.append(sep);
-                sep = "";
-                String str = Long.toHexString(i);
-                for (int j = str.length(); j < 8; j++)
-                    builder.append('0');
-                builder.append(str);
-                for (int j = 0; j < width; j++) {
-                    if (j == width / 2)
-                        builder.append(' ');
-                    if (i + j < offset || i + j >= offset + len) {
-                        builder.append("   ");
-
-                    } else {
-                        builder.append(' ');
-                        int ch = bytes.readUnsignedByte(i + j);
-                        builder.append(HEXI_DECIMAL[ch >> 4]);
-                        builder.append(HEXI_DECIMAL[ch & 15]);
-                    }
-                }
-                builder.append(' ');
-                for (int j = 0; j < width; j++) {
-                    if (j == width / 2)
-                        builder.append(' ');
-                    if (i + j < offset || i + j >= offset + len) {
-                        builder.append(' ');
-
-                    } else {
-                        int ch = bytes.readUnsignedByte(i + j);
-                        if (ch < ' ' || ch > 126)
-                            ch = '\u00B7';
-                        builder.append((char) ch);
-                    }
-                }
-                builder.append("\n");
-            }
-            return builder.toString();
-        } finally {
-            bytes.readLimit(limit);
-            bytes.readPosition(position);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/FQLQuery.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/FQLQuery.java b/src/java/org/apache/cassandra/tools/fqltool/FQLQuery.java
new file mode 100644
index 0000000..6c0a6b9
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/FQLQuery.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import com.google.common.primitives.Longs;
+
+import com.datastax.driver.core.BatchStatement;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.audit.FullQueryLogger;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.binlog.BinLog;
+
+public abstract class FQLQuery implements Comparable<FQLQuery>
+{
+    public final long queryStartTime;
+    public final QueryOptions queryOptions;
+    public final int protocolVersion;
+    public final String keyspace;
+    public final long generatedTimestamp;
+    private final int generatedNowInSeconds;
+
+    public FQLQuery(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds)
+    {
+        this.queryStartTime = queryStartTime;
+        this.queryOptions = queryOptions;
+        this.protocolVersion = protocolVersion;
+        this.keyspace = keyspace;
+        this.generatedTimestamp = generatedTimestamp;
+        this.generatedNowInSeconds = generatedNowInSeconds;
+    }
+
+    public abstract Statement toStatement();
+
+    /**
+     * used when storing the queries executed
+     */
+    public abstract BinLog.ReleaseableWriteMarshallable toMarshallable();
+
+    public QueryState queryState()
+    {
+        ClientState clientState = keyspace != null ? ClientState.forInternalCalls(keyspace) : ClientState.forInternalCalls();
+
+        return new QueryState(clientState, generatedTimestamp, generatedNowInSeconds);
+    }
+
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (!(o instanceof FQLQuery)) return false;
+        FQLQuery fqlQuery = (FQLQuery) o;
+        return queryStartTime == fqlQuery.queryStartTime &&
+               protocolVersion == fqlQuery.protocolVersion &&
+               generatedTimestamp == fqlQuery.generatedTimestamp &&
+               generatedNowInSeconds == fqlQuery.generatedNowInSeconds &&
+               Objects.equals(queryOptions.getValues(), fqlQuery.queryOptions.getValues()) &&
+               Objects.equals(keyspace, fqlQuery.keyspace);
+    }
+
+    public int hashCode()
+    {
+        return Objects.hash(queryStartTime, queryOptions, protocolVersion, keyspace, generatedTimestamp, generatedNowInSeconds);
+    }
+
+    public int compareTo(FQLQuery other)
+    {
+        int cmp = Longs.compare(queryStartTime, other.queryStartTime);
+        if (cmp != 0)
+            return cmp;
+        cmp = Longs.compare(generatedTimestamp, other.generatedTimestamp);
+        if (cmp != 0)
+            return cmp;
+
+        return Longs.compare(generatedNowInSeconds, other.generatedNowInSeconds);
+    }
+
+    public String toString()
+    {
+        return "FQLQuery{" +
+               "queryStartTime=" + queryStartTime +
+               ", protocolVersion=" + protocolVersion +
+               ", keyspace='" + keyspace + '\'' +
+               ", generatedTimestamp=" + generatedTimestamp +
+               ", generatedNowInSeconds=" + generatedNowInSeconds +
+               '}';
+    }
+
+    public static class Single extends FQLQuery
+    {
+        public final String query;
+        public final List<ByteBuffer> values;
+
+        public Single(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds, String queryString, List<ByteBuffer> values)
+        {
+            super(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds);
+            this.query = queryString;
+            this.values = values;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("%s%nQuery = %s, Values = %s",
+                                 super.toString(),
+                                 query,
+                                 values.stream().map(ByteBufferUtil::bytesToHex).collect(Collectors.joining(",")));
+        }
+
+        public Statement toStatement()
+        {
+            SimpleStatement ss = new SimpleStatement(query, values.toArray());
+            ss.setConsistencyLevel(ConsistencyLevel.valueOf(queryOptions.getConsistency().name()));
+            ss.setDefaultTimestamp(generatedTimestamp);
+            return ss;
+        }
+
+        public BinLog.ReleaseableWriteMarshallable toMarshallable()
+        {
+
+            return new FullQueryLogger.Query(query, queryOptions, queryState(), queryStartTime);
+        }
+
+        public int compareTo(FQLQuery other)
+        {
+            int cmp = super.compareTo(other);
+
+            if (cmp == 0)
+            {
+                if (other instanceof Batch)
+                    return -1;
+
+                Single singleQuery = (Single) other;
+
+                cmp = query.compareTo(singleQuery.query);
+                if (cmp == 0)
+                {
+                    if (values.size() != singleQuery.values.size())
+                        return values.size() - singleQuery.values.size();
+                    for (int i = 0; i < values.size(); i++)
+                    {
+                        cmp = values.get(i).compareTo(singleQuery.values.get(i));
+                        if (cmp != 0)
+                            return cmp;
+                    }
+                }
+            }
+            return cmp;
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (!(o instanceof Single)) return false;
+            if (!super.equals(o)) return false;
+            Single single = (Single) o;
+            return Objects.equals(query, single.query) &&
+                   Objects.equals(values, single.values);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(super.hashCode(), query, values);
+        }
+    }
+
+    public static class Batch extends FQLQuery
+    {
+        public final BatchStatement.Type batchType;
+        public final List<Single> queries;
+
+        public Batch(String keyspace, int protocolVersion, QueryOptions queryOptions, long queryStartTime, long generatedTimestamp, int generatedNowInSeconds, BatchStatement.Type batchType, List<String> queries, List<List<ByteBuffer>> values)
+        {
+            super(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds);
+            this.batchType = batchType;
+            this.queries = new ArrayList<>(queries.size());
+            for (int i = 0; i < queries.size(); i++)
+                this.queries.add(new Single(keyspace, protocolVersion, queryOptions, queryStartTime, generatedTimestamp, generatedNowInSeconds, queries.get(i), values.get(i)));
+        }
+
+        public Statement toStatement()
+        {
+            BatchStatement bs = new BatchStatement(batchType);
+
+            for (Single query : queries)
+            {
+                bs.add(new SimpleStatement(query.query, query.values.toArray()));
+            }
+            bs.setConsistencyLevel(ConsistencyLevel.valueOf(queryOptions.getConsistency().name()));
+            bs.setDefaultTimestamp(generatedTimestamp); // todo: set actual server side generated time
+            return bs;
+        }
+
+        public int compareTo(FQLQuery other)
+        {
+            int cmp = super.compareTo(other);
+
+            if (cmp == 0)
+            {
+                if (other instanceof Single)
+                    return 1;
+
+                Batch otherBatch = (Batch) other;
+                if (queries.size() != otherBatch.queries.size())
+                    return queries.size() - otherBatch.queries.size();
+                for (int i = 0; i < queries.size(); i++)
+                {
+                    cmp = queries.get(i).compareTo(otherBatch.queries.get(i));
+                    if (cmp != 0)
+                        return cmp;
+                }
+            }
+            return cmp;
+        }
+
+        public BinLog.ReleaseableWriteMarshallable toMarshallable()
+        {
+            List<String> queryStrings = new ArrayList<>();
+            List<List<ByteBuffer>> values = new ArrayList<>();
+            for (Single q : queries)
+            {
+                queryStrings.add(q.query);
+                values.add(q.values);
+            }
+            return new FullQueryLogger.Batch(org.apache.cassandra.cql3.statements.BatchStatement.Type.valueOf(batchType.name()), queryStrings, values, queryOptions, queryState(), queryStartTime);
+        }
+
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder(super.toString()).append("\nbatch: ").append(batchType).append('\n');
+            for (Single q : queries)
+                sb.append(q.toString()).append('\n');
+            sb.append("end batch");
+            return sb.toString();
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (!(o instanceof Batch)) return false;
+            if (!super.equals(o)) return false;
+            Batch batch = (Batch) o;
+            return batchType == batch.batchType &&
+                   Objects.equals(queries, batch.queries);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(super.hashCode(), batchType, queries);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/FQLQueryIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/FQLQueryIterator.java b/src/java/org/apache/cassandra/tools/fqltool/FQLQueryIterator.java
new file mode 100644
index 0000000..390a52e
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/FQLQueryIterator.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+import java.util.PriorityQueue;
+
+import net.openhft.chronicle.queue.ExcerptTailer;
+import org.apache.cassandra.utils.AbstractIterator;
+
+public class FQLQueryIterator extends AbstractIterator<FQLQuery>
+{
+    // use a priority queue to be able to sort the head of the query logs in memory
+    private final PriorityQueue<FQLQuery> pq;
+    private final ExcerptTailer tailer;
+    private final FQLQueryReader reader;
+
+    /**
+     * Create an iterator over the FQLQueries in tailer
+     *
+     * Reads up to readAhead queries in to memory to be able to sort them (the files are mostly sorted already)
+     */
+    public FQLQueryIterator(ExcerptTailer tailer, int readAhead)
+    {
+        assert readAhead > 0 : "readAhead needs to be > 0";
+        reader = new FQLQueryReader();
+        this.tailer = tailer;
+        pq = new PriorityQueue<>(readAhead);
+        for (int i = 0; i < readAhead; i++)
+        {
+            FQLQuery next = readNext();
+            if (next != null)
+                pq.add(next);
+            else
+                break;
+        }
+    }
+
+    protected FQLQuery computeNext()
+    {
+        FQLQuery q = pq.poll();
+        if (q == null)
+            return endOfData();
+        FQLQuery next = readNext();
+        if (next != null)
+            pq.add(next);
+        return q;
+    }
+
+    private FQLQuery readNext()
+    {
+        if (tailer.readDocument(reader))
+            return reader.getQuery();
+        return null;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/FQLQueryReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/FQLQueryReader.java b/src/java/org/apache/cassandra/tools/fqltool/FQLQueryReader.java
new file mode 100644
index 0000000..af77c59
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/FQLQueryReader.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.datastax.driver.core.BatchStatement;
+import io.netty.buffer.Unpooled;
+import net.openhft.chronicle.core.io.IORuntimeException;
+import net.openhft.chronicle.wire.ReadMarshallable;
+import net.openhft.chronicle.wire.ValueIn;
+import net.openhft.chronicle.wire.WireIn;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static org.apache.cassandra.audit.FullQueryLogger.GENERATED_NOW_IN_SECONDS;
+import static org.apache.cassandra.audit.FullQueryLogger.GENERATED_TIMESTAMP;
+import static org.apache.cassandra.audit.FullQueryLogger.KEYSPACE;
+import static org.apache.cassandra.audit.FullQueryLogger.PROTOCOL_VERSION;
+import static org.apache.cassandra.audit.FullQueryLogger.QUERY_OPTIONS;
+import static org.apache.cassandra.audit.FullQueryLogger.QUERY_START_TIME;
+import static org.apache.cassandra.audit.FullQueryLogger.TYPE;
+import static org.apache.cassandra.audit.FullQueryLogger.VERSION;
+import static org.apache.cassandra.audit.FullQueryLogger.BATCH;
+import static org.apache.cassandra.audit.FullQueryLogger.BATCH_TYPE;
+import static org.apache.cassandra.audit.FullQueryLogger.QUERIES;
+import static org.apache.cassandra.audit.FullQueryLogger.QUERY;
+import static org.apache.cassandra.audit.FullQueryLogger.SINGLE_QUERY;
+import static org.apache.cassandra.audit.FullQueryLogger.VALUES;
+
+public class FQLQueryReader implements ReadMarshallable
+{
+    private FQLQuery query;
+
+    public void readMarshallable(WireIn wireIn) throws IORuntimeException
+    {
+        int currentVersion = wireIn.read(VERSION).int16();
+        String type = wireIn.read(TYPE).text();
+        long queryStartTime = wireIn.read(QUERY_START_TIME).int64();
+        int protocolVersion = wireIn.read(PROTOCOL_VERSION).int32();
+        QueryOptions queryOptions = QueryOptions.codec.decode(Unpooled.wrappedBuffer(wireIn.read(QUERY_OPTIONS).bytes()), ProtocolVersion.decode(protocolVersion));
+        long generatedTimestamp = wireIn.read(GENERATED_TIMESTAMP).int64();
+        int generatedNowInSeconds = wireIn.read(GENERATED_NOW_IN_SECONDS).int32();
+        String keyspace = wireIn.read(KEYSPACE).text();
+
+        switch (type)
+        {
+            case SINGLE_QUERY:
+                String queryString = wireIn.read(QUERY).text();
+                query = new FQLQuery.Single(keyspace,
+                                            protocolVersion,
+                                            queryOptions,
+                                            queryStartTime,
+                                            generatedTimestamp,
+                                            generatedNowInSeconds,
+                                            queryString,
+                                            queryOptions.getValues());
+                break;
+            case BATCH:
+                BatchStatement.Type batchType = BatchStatement.Type.valueOf(wireIn.read(BATCH_TYPE).text());
+                ValueIn in = wireIn.read(QUERIES);
+                int queryCount = in.int32();
+
+                List<String> queries = new ArrayList<>(queryCount);
+                for (int i = 0; i < queryCount; i++)
+                    queries.add(in.text());
+                in = wireIn.read(VALUES);
+                int valueCount = in.int32();
+                List<List<ByteBuffer>> values = new ArrayList<>(valueCount);
+                for (int ii = 0; ii < valueCount; ii++)
+                {
+                    List<ByteBuffer> subValues = new ArrayList<>();
+                    values.add(subValues);
+                    int numSubValues = in.int32();
+                    for (int zz = 0; zz < numSubValues; zz++)
+                        subValues.add(ByteBuffer.wrap(in.bytes()));
+                }
+                query = new FQLQuery.Batch(keyspace,
+                                           protocolVersion,
+                                           queryOptions,
+                                           queryStartTime,
+                                           generatedTimestamp,
+                                           generatedNowInSeconds,
+                                           batchType,
+                                           queries,
+                                           values);
+                break;
+            default:
+                throw new RuntimeException("Unknown type: " + type);
+        }
+    }
+
+    public FQLQuery getQuery()
+    {
+        return query;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/QueryReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/QueryReplayer.java b/src/java/org/apache/cassandra/tools/fqltool/QueryReplayer.java
new file mode 100644
index 0000000..0c8382f
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/QueryReplayer.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.util.concurrent.FluentFuture;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class QueryReplayer implements Closeable
+{
+    private static final int PRINT_RATE = 5000;
+    private final ExecutorService es = Executors.newFixedThreadPool(1);
+    private final Iterator<List<FQLQuery>> queryIterator;
+    private final List<Cluster> targetClusters;
+    private final List<Predicate<FQLQuery>> filters;
+    private final List<Session> sessions;
+    private final ResultHandler resultHandler;
+    private final MetricRegistry metrics = new MetricRegistry();
+    private final boolean debug;
+    private final PrintStream out;
+
+    public QueryReplayer(Iterator<List<FQLQuery>> queryIterator,
+                         List<String> targetHosts,
+                         List<File> resultPaths,
+                         List<Predicate<FQLQuery>> filters,
+                         PrintStream out,
+                         String queryFilePathString,
+                         boolean debug)
+    {
+        this.queryIterator = queryIterator;
+        targetClusters = targetHosts.stream().map(h -> Cluster.builder().addContactPoint(h).build()).collect(Collectors.toList());
+        this.filters = filters;
+        sessions = targetClusters.stream().map(Cluster::connect).collect(Collectors.toList());
+        File queryFilePath = queryFilePathString != null ? new File(queryFilePathString) : null;
+        resultHandler = new ResultHandler(targetHosts, resultPaths, queryFilePath);
+        this.debug = debug;
+        this.out = out;
+    }
+
+    public void replay()
+    {
+        while (queryIterator.hasNext())
+        {
+            List<FQLQuery> queries = queryIterator.next();
+            for (FQLQuery query : queries)
+            {
+                if (filters.stream().anyMatch(f -> !f.test(query)))
+                    continue;
+                try (Timer.Context ctx = metrics.timer("queries").time())
+                {
+                    List<ListenableFuture<ResultHandler.ComparableResultSet>> results = new ArrayList<>(sessions.size());
+                    Statement statement = query.toStatement();
+                    for (Session session : sessions)
+                    {
+                        try
+                        {
+                            if (query.keyspace != null && !query.keyspace.equals(session.getLoggedKeyspace()))
+                            {
+                                if (debug)
+                                    out.printf("Switching keyspace from %s to %s%n", session.getLoggedKeyspace(), query.keyspace);
+                                session.execute("USE " + query.keyspace);
+                            }
+                        }
+                        catch (Throwable t)
+                        {
+                            out.printf("USE %s failed: %s%n", query.keyspace, t.getMessage());
+                        }
+                        if (debug)
+                        {
+                            out.println("Executing query:");
+                            out.println(query);
+                        }
+                        ListenableFuture<ResultSet> future = session.executeAsync(statement);
+                        results.add(handleErrors(future));
+                    }
+
+                    ListenableFuture<List<ResultHandler.ComparableResultSet>> resultList = Futures.allAsList(results);
+
+                    Futures.addCallback(resultList, new FutureCallback<List<ResultHandler.ComparableResultSet>>()
+                    {
+                        public void onSuccess(@Nullable List<ResultHandler.ComparableResultSet> resultSets)
+                        {
+                            // note that the order of resultSets is signifcant here - resultSets.get(x) should
+                            // be the result from a query against targetHosts.get(x)
+                            resultHandler.handleResults(query, resultSets);
+                        }
+
+                        public void onFailure(Throwable throwable)
+                        {
+                            throw new AssertionError("Errors should be handled in FQLQuery.execute", throwable);
+                        }
+                    }, es);
+
+                    FBUtilities.waitOnFuture(resultList);
+                }
+                catch (Throwable t)
+                {
+                    out.printf("QUERY %s got exception: %s", query, t.getMessage());
+                }
+
+                Timer timer = metrics.timer("queries");
+                if (timer.getCount() % PRINT_RATE == 0)
+                    out.printf("%d queries, rate = %.2f%n", timer.getCount(), timer.getOneMinuteRate());
+            }
+        }
+    }
+
+    /**
+     * Make sure we catch any query errors
+     *
+     * On error, this creates a failed ComparableResultSet with the exception set to be able to store
+     * this fact in the result file and handle comparison of failed result sets.
+     */
+    private static ListenableFuture<ResultHandler.ComparableResultSet> handleErrors(ListenableFuture<ResultSet> result)
+    {
+        FluentFuture<ResultHandler.ComparableResultSet> fluentFuture = FluentFuture.from(result)
+                                                                                   .transform(DriverResultSet::new, MoreExecutors.directExecutor());
+        return fluentFuture.catching(Throwable.class, DriverResultSet::failed, MoreExecutors.directExecutor());
+    }
+
+    public void close()
+    {
+        sessions.forEach(Session::close);
+        targetClusters.forEach(Cluster::close);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/ResultComparator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/ResultComparator.java b/src/java/org/apache/cassandra/tools/fqltool/ResultComparator.java
new file mode 100644
index 0000000..4bbaf7a
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/ResultComparator.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Streams;
+
+public class ResultComparator
+{
+    /**
+     * Compares the rows in rows
+     * the row at position x in rows will have come from host at position x in targetHosts
+     */
+    public boolean compareRows(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableRow> rows)
+    {
+        if (rows.size() < 2 || rows.stream().allMatch(Objects::isNull))
+            return true;
+
+        if (rows.stream().anyMatch(Objects::isNull))
+        {
+            handleMismatch(targetHosts, query, rows);
+            return false;
+        }
+
+        ResultHandler.ComparableRow ref = rows.get(0);
+        boolean equal = true;
+        for (int i = 1; i < rows.size(); i++)
+        {
+            ResultHandler.ComparableRow compare = rows.get(i);
+            if (!ref.equals(compare))
+                equal = false;
+        }
+        if (!equal)
+            handleMismatch(targetHosts, query, rows);
+        return equal;
+    }
+
+    /**
+     * Compares the column definitions
+     *
+     * the column definitions at position x in cds will have come from host at position x in targetHosts
+     */
+    public boolean compareColumnDefinitions(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
+    {
+        if (cds.size() < 2)
+            return true;
+
+        boolean equal = true;
+        List<ResultHandler.ComparableDefinition> refDefs = cds.get(0).asList();
+        for (int i = 1; i < cds.size(); i++)
+        {
+            List<ResultHandler.ComparableDefinition> toCompare = cds.get(i).asList();
+            if (!refDefs.equals(toCompare))
+                equal = false;
+        }
+        if (!equal)
+            handleColumnDefMismatch(targetHosts, query, cds);
+        return equal;
+    }
+
+    private void handleMismatch(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableRow> rows)
+    {
+        System.out.println("MISMATCH:");
+        System.out.println("Query = " + query);
+        System.out.println("Results:");
+        System.out.println(Streams.zip(rows.stream(), targetHosts.stream(), (r, host) -> String.format("%s: %s%n", host, r == null ? "null" : r)).collect(Collectors.joining()));
+    }
+
+    private void handleColumnDefMismatch(List<String> targetHosts, FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
+    {
+        System.out.println("COLUMN DEFINITION MISMATCH:");
+        System.out.println("Query = " + query);
+        System.out.println("Results: ");
+        System.out.println(Streams.zip(cds.stream(), targetHosts.stream(), (cd, host) -> String.format("%s: %s%n", host, columnDefinitionsString(cd))).collect(Collectors.joining()));
+    }
+
+    private String columnDefinitionsString(ResultHandler.ComparableColumnDefinitions cd)
+    {
+        StringBuilder sb = new StringBuilder();
+        if (cd == null)
+            sb.append("NULL");
+        else if (cd.wasFailed())
+            sb.append("FAILED");
+        else
+        {
+            for (ResultHandler.ComparableDefinition def : cd)
+            {
+                sb.append(def.toString());
+            }
+        }
+        return sb.toString();
+    }
+
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/ResultHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/ResultHandler.java b/src/java/org/apache/cassandra/tools/fqltool/ResultHandler.java
new file mode 100644
index 0000000..c769231
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/ResultHandler.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class ResultHandler
+{
+    private final ResultStore resultStore;
+    private final ResultComparator resultComparator;
+    private final List<String> targetHosts;
+
+    public ResultHandler(List<String> targetHosts, List<File> resultPaths, File queryFilePath)
+    {
+        this.targetHosts = targetHosts;
+        resultStore = resultPaths != null ? new ResultStore(resultPaths, queryFilePath) : null;
+        resultComparator = new ResultComparator();
+    }
+
+    /**
+     * Since we can't iterate a ResultSet more than once, and we don't want to keep the entire result set in memory
+     * we feed the rows one-by-one to resultComparator and resultStore.
+     *
+     * results.get(x) should be the results from executing query against targetHosts.get(x)
+     */
+    public void handleResults(FQLQuery query, List<ComparableResultSet> results)
+    {
+        for (int i = 0; i < targetHosts.size(); i++)
+        {
+            if (results.get(i).wasFailed())
+            {
+                System.out.println("Query against "+targetHosts.get(i)+" failure:");
+                System.out.println(query);
+                System.out.println("Message: "+results.get(i).getFailureException().getMessage());
+            }
+        }
+
+        List<ComparableColumnDefinitions> columnDefinitions = results.stream().map(ComparableResultSet::getColumnDefinitions).collect(Collectors.toList());
+        resultComparator.compareColumnDefinitions(targetHosts, query, columnDefinitions);
+        if (resultStore != null)
+            resultStore.storeColumnDefinitions(query, columnDefinitions);
+        List<Iterator<ComparableRow>> iters = results.stream().map(Iterable::iterator).collect(Collectors.toList());
+
+        while (true)
+        {
+            List<ComparableRow> rows = rows(iters);
+            resultComparator.compareRows(targetHosts, query, rows);
+            if (resultStore != null)
+                resultStore.storeRows(rows);
+            // all rows being null marks end of all resultsets, we need to call compareRows
+            // and storeRows once with everything null to mark that fact
+            if (rows.stream().allMatch(Objects::isNull))
+                return;
+        }
+    }
+
+    /**
+     * Get the first row from each of the iterators, if the iterator has run out, null will mark that in the list
+     */
+    @VisibleForTesting
+    public static List<ComparableRow> rows(List<Iterator<ComparableRow>> iters)
+    {
+        List<ComparableRow> rows = new ArrayList<>(iters.size());
+        for (Iterator<ComparableRow> iter : iters)
+        {
+            if (iter.hasNext())
+                rows.add(iter.next());
+            else
+                rows.add(null);
+        }
+        return rows;
+    }
+
+    public interface ComparableResultSet extends Iterable<ComparableRow>
+    {
+        public ComparableColumnDefinitions getColumnDefinitions();
+        public boolean wasFailed();
+        public Throwable getFailureException();
+    }
+
+    public interface ComparableColumnDefinitions extends Iterable<ComparableDefinition>
+    {
+        public List<ComparableDefinition> asList();
+        public boolean wasFailed();
+        public int size();
+    }
+
+    public interface ComparableDefinition
+    {
+        public String getType();
+        public String getName();
+    }
+
+    public interface ComparableRow
+    {
+        public ByteBuffer getBytesUnsafe(int i);
+        public ComparableColumnDefinitions getColumnDefinitions();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/ResultStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/ResultStore.java b/src/java/org/apache/cassandra/tools/fqltool/ResultStore.java
new file mode 100644
index 0000000..6d6aaac
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/ResultStore.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import net.openhft.chronicle.bytes.BytesStore;
+import net.openhft.chronicle.core.io.Closeable;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+import net.openhft.chronicle.queue.ExcerptAppender;
+import net.openhft.chronicle.wire.ValueOut;
+import org.apache.cassandra.utils.binlog.BinLog;
+
+/**
+ * see FQLReplayTest#readResultFile for how to read files produced by this class
+ */
+public class ResultStore
+{
+    private final List<ChronicleQueue> queues;
+    private final List<ExcerptAppender> appenders;
+    private final ChronicleQueue queryStoreQueue;
+    private final ExcerptAppender queryStoreAppender;
+    private final Set<Integer> finishedHosts = new HashSet<>();
+
+    public ResultStore(List<File> resultPaths, File queryFilePath)
+    {
+        queues = resultPaths.stream().map(path -> ChronicleQueueBuilder.single(path).build()).collect(Collectors.toList());
+        appenders = queues.stream().map(ChronicleQueue::acquireAppender).collect(Collectors.toList());
+        queryStoreQueue = queryFilePath != null ? ChronicleQueueBuilder.single(queryFilePath).build() : null;
+        queryStoreAppender = queryStoreQueue != null ? queryStoreQueue.acquireAppender() : null;
+    }
+
+    /**
+     * Store the column definitions in cds
+     *
+     * the ColumnDefinitions at position x will get stored by the appender at position x
+     *
+     * Calling this method indicates that we are starting a new result set from a query, it must be called before
+     * calling storeRows.
+     *
+     */
+    public void storeColumnDefinitions(FQLQuery query, List<ResultHandler.ComparableColumnDefinitions> cds)
+    {
+        finishedHosts.clear();
+        if (queryStoreAppender != null)
+        {
+            BinLog.ReleaseableWriteMarshallable writeMarshallableQuery = query.toMarshallable();
+            queryStoreAppender.writeDocument(writeMarshallableQuery);
+            writeMarshallableQuery.release();
+        }
+        for (int i = 0; i < cds.size(); i++)
+        {
+            ResultHandler.ComparableColumnDefinitions cd = cds.get(i);
+            appenders.get(i).writeDocument(wire ->
+                                           {
+                                               if (!cd.wasFailed())
+                                               {
+                                                   wire.write("type").text("column_definitions");
+                                                   wire.write("column_count").int32(cd.size());
+                                                   for (ResultHandler.ComparableDefinition d : cd.asList())
+                                                   {
+                                                       ValueOut vo = wire.write("column_definition");
+                                                       vo.text(d.getName());
+                                                       vo.text(d.getType());
+                                                   }
+                                               }
+                                               else
+                                               {
+                                                   wire.write("type").text("query_failed");
+                                               }
+                                           });
+        }
+    }
+
+    /**
+     * Store rows
+     *
+     * the row at position x will get stored by appender at position x
+     *
+     * Before calling this for a new result set, storeColumnDefinitions must be called.
+     */
+    public void storeRows(List<ResultHandler.ComparableRow> rows)
+    {
+        for (int i = 0; i < rows.size(); i++)
+        {
+            ResultHandler.ComparableRow row = rows.get(i);
+            if (row == null && !finishedHosts.contains(i))
+            {
+                appenders.get(i).writeDocument(wire -> wire.write("type").text("end_resultset"));
+                finishedHosts.add(i);
+            }
+            else if (row != null)
+            {
+                appenders.get(i).writeDocument(wire ->
+                                               {
+                                                   {
+                                                       wire.write("type").text("row");
+                                                       wire.write("row_column_count").int32(row.getColumnDefinitions().size());
+                                                       for (int jj = 0; jj < row.getColumnDefinitions().size(); jj++)
+                                                       {
+                                                           ByteBuffer bb = row.getBytesUnsafe(jj);
+                                                           if (bb != null)
+                                                               wire.write("column").bytes(BytesStore.wrap(bb));
+                                                           else
+                                                               wire.write("column").bytes("NULL".getBytes());
+                                                       }
+                                                   }
+                                               });
+            }
+        }
+    }
+
+    public void close()
+    {
+        queues.forEach(Closeable::close);
+        if (queryStoreQueue != null)
+            queryStoreQueue.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/commands/Dump.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/commands/Dump.java b/src/java/org/apache/cassandra/tools/fqltool/commands/Dump.java
new file mode 100644
index 0000000..5c23d3e
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/commands/Dump.java
@@ -0,0 +1,325 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool.commands;
+
+import java.io.File;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import io.airlift.airline.Option;
+import io.netty.buffer.Unpooled;
+import net.openhft.chronicle.bytes.Bytes;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+import net.openhft.chronicle.queue.ExcerptTailer;
+import net.openhft.chronicle.queue.RollCycles;
+import net.openhft.chronicle.threads.Pauser;
+import net.openhft.chronicle.wire.ReadMarshallable;
+import net.openhft.chronicle.wire.ValueIn;
+import net.openhft.chronicle.wire.WireIn;
+import org.apache.cassandra.audit.FullQueryLogger;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * Dump the contents of a list of paths containing full query logs
+ */
+@Command(name = "dump", description = "Dump the contents of a full query log")
+public class Dump implements Runnable
+{
+    static final char[] HEXI_DECIMAL = "0123456789ABCDEF".toCharArray();
+
+    @Arguments(usage = "<path1> [<path2>...<pathN>]", description = "Path containing the full query logs to dump.", required = true)
+    private List<String> arguments = new ArrayList<>();
+
+    @Option(title = "roll_cycle", name = {"--roll-cycle"}, description = "How often to roll the log file was rolled. May be necessary for Chronicle to correctly parse file names. (MINUTELY, HOURLY, DAILY). Default HOURLY.")
+    private String rollCycle = "HOURLY";
+
+    @Option(title = "follow", name = {"--follow"}, description = "Upon reacahing the end of the log continue indefinitely waiting for more records")
+    private boolean follow = false;
+
+    @Override
+    public void run()
+    {
+        dump(arguments, rollCycle, follow);
+    }
+
+    public static void dump(List<String> arguments, String rollCycle, boolean follow)
+    {
+        StringBuilder sb = new StringBuilder();
+        ReadMarshallable reader = wireIn ->
+        {
+            sb.setLength(0);
+
+            int version = wireIn.read(FullQueryLogger.VERSION).int16();
+            if (version != FullQueryLogger.CURRENT_VERSION)
+                throw new UnsupportedOperationException("Full query log of unexpected version " + version + " encountered");
+
+            String type = wireIn.read(FullQueryLogger.TYPE).text();
+            sb.append("Type: ")
+              .append(type)
+              .append(System.lineSeparator());
+
+            long queryStartTime = wireIn.read(FullQueryLogger.QUERY_START_TIME).int64();
+            sb.append("Query start time: ")
+              .append(queryStartTime)
+              .append(System.lineSeparator());
+
+            int protocolVersion = wireIn.read(FullQueryLogger.PROTOCOL_VERSION).int32();
+            sb.append("Protocol version: ")
+              .append(protocolVersion)
+              .append(System.lineSeparator());
+
+            QueryOptions options =
+                QueryOptions.codec.decode(Unpooled.wrappedBuffer(wireIn.read(FullQueryLogger.QUERY_OPTIONS).bytes()),
+                                          ProtocolVersion.decode(protocolVersion));
+
+            long generatedTimestamp = wireIn.read(FullQueryLogger.GENERATED_TIMESTAMP).int64();
+            sb.append("Generated timestamp:")
+              .append(generatedTimestamp)
+              .append(System.lineSeparator());
+
+            int generatedNowInSeconds = wireIn.read(FullQueryLogger.GENERATED_NOW_IN_SECONDS).int32();
+            sb.append("Generated nowInSeconds:")
+              .append(generatedNowInSeconds)
+              .append(System.lineSeparator());
+
+            switch (type)
+            {
+                case (FullQueryLogger.SINGLE_QUERY):
+                    dumpQuery(options, wireIn, sb);
+                    break;
+
+                case (FullQueryLogger.BATCH):
+                    dumpBatch(options, wireIn, sb);
+                    break;
+
+                default:
+                    throw new UnsupportedOperationException("Log entry of unsupported type " + type);
+            }
+
+            System.out.print(sb.toString());
+            System.out.flush();
+        };
+
+        //Backoff strategy for spinning on the queue, not aggressive at all as this doesn't need to be low latency
+        Pauser pauser = Pauser.millis(100);
+        List<ChronicleQueue> queues = arguments.stream().distinct().map(path -> ChronicleQueueBuilder.single(new File(path)).readOnly(true).rollCycle(RollCycles.valueOf(rollCycle)).build()).collect(Collectors.toList());
+        List<ExcerptTailer> tailers = queues.stream().map(ChronicleQueue::createTailer).collect(Collectors.toList());
+        boolean hadWork = true;
+        while (hadWork)
+        {
+            hadWork = false;
+            for (ExcerptTailer tailer : tailers)
+            {
+                while (tailer.readDocument(reader))
+                {
+                    hadWork = true;
+                }
+            }
+
+            if (follow)
+            {
+                if (!hadWork)
+                {
+                    //Chronicle queue doesn't support blocking so use this backoff strategy
+                    pauser.pause();
+                }
+                //Don't terminate the loop even if there wasn't work
+                hadWork = true;
+            }
+        }
+    }
+
+    private static void dumpQuery(QueryOptions options, WireIn wireIn, StringBuilder sb)
+    {
+        sb.append("Query: ")
+          .append(wireIn.read(FullQueryLogger.QUERY).text())
+          .append(System.lineSeparator());
+
+        List<ByteBuffer> values = options.getValues() != null
+                                ? options.getValues()
+                                : Collections.emptyList();
+
+        sb.append("Values: ")
+          .append(System.lineSeparator());
+        appendValuesToStringBuilder(values, sb);
+        sb.append(System.lineSeparator());
+    }
+
+    private static void dumpBatch(QueryOptions options, WireIn wireIn, StringBuilder sb)
+    {
+        sb.append("Batch type: ")
+          .append(wireIn.read(FullQueryLogger.BATCH_TYPE).text())
+          .append(System.lineSeparator());
+
+        ValueIn in = wireIn.read(FullQueryLogger.QUERIES);
+        int numQueries = in.int32();
+        List<String> queries = new ArrayList<>(numQueries);
+        for (int i = 0; i < numQueries; i++)
+            queries.add(in.text());
+
+        in = wireIn.read(FullQueryLogger.VALUES);
+        int numValues = in.int32();
+
+        for (int i = 0; i < numValues; i++)
+        {
+            int numSubValues = in.int32();
+            List<ByteBuffer> subValues = new ArrayList<>(numSubValues);
+            for (int j = 0; j < numSubValues; j++)
+                subValues.add(ByteBuffer.wrap(in.bytes()));
+
+            sb.append("Query: ")
+              .append(queries.get(i))
+              .append(System.lineSeparator());
+
+            sb.append("Values: ")
+              .append(System.lineSeparator());
+            appendValuesToStringBuilder(subValues, sb);
+        }
+
+        sb.append(System.lineSeparator());
+    }
+
+    private static void appendValuesToStringBuilder(List<ByteBuffer> values, StringBuilder sb)
+    {
+        boolean first = true;
+        for (ByteBuffer value : values)
+        {
+            Bytes bytes = Bytes.wrapForRead(value);
+            long maxLength2 = Math.min(1024, bytes.readLimit() - bytes.readPosition());
+            toHexString(bytes, bytes.readPosition(), maxLength2, sb);
+            if (maxLength2 < bytes.readLimit() - bytes.readPosition())
+            {
+                sb.append("... truncated").append(System.lineSeparator());
+            }
+
+            if (first)
+            {
+                first = false;
+            }
+            else
+            {
+                sb.append("-----").append(System.lineSeparator());
+            }
+        }
+    }
+
+    //This is from net.openhft.chronicle.bytes, need to pass in the StringBuilder so had to copy
+    /*
+     * Copyright 2016 higherfrequencytrading.com
+     *
+     * Licensed 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.
+     */
+    /**
+     * display the hex data of {@link Bytes} from the position() to the limit()
+     *
+     * @param bytes the buffer you wish to toString()
+     * @return hex representation of the buffer, from example [0D ,OA, FF]
+     */
+    public static String toHexString(final Bytes bytes, long offset, long len, StringBuilder builder)
+    throws BufferUnderflowException
+    {
+        if (len == 0)
+            return "";
+
+        int width = 16;
+        int[] lastLine = new int[width];
+        String sep = "";
+        long position = bytes.readPosition();
+        long limit = bytes.readLimit();
+
+        try {
+            bytes.readPositionRemaining(offset, len);
+
+            long start = offset / width * width;
+            long end = (offset + len + width - 1) / width * width;
+            for (long i = start; i < end; i += width) {
+                // check for duplicate rows
+                if (i + width < end) {
+                    boolean same = true;
+
+                    for (int j = 0; j < width && i + j < offset + len; j++) {
+                        int ch = bytes.readUnsignedByte(i + j);
+                        same &= (ch == lastLine[j]);
+                        lastLine[j] = ch;
+                    }
+                    if (i > start && same) {
+                        sep = "........\n";
+                        continue;
+                    }
+                }
+                builder.append(sep);
+                sep = "";
+                String str = Long.toHexString(i);
+                for (int j = str.length(); j < 8; j++)
+                    builder.append('0');
+                builder.append(str);
+                for (int j = 0; j < width; j++) {
+                    if (j == width / 2)
+                        builder.append(' ');
+                    if (i + j < offset || i + j >= offset + len) {
+                        builder.append("   ");
+
+                    } else {
+                        builder.append(' ');
+                        int ch = bytes.readUnsignedByte(i + j);
+                        builder.append(HEXI_DECIMAL[ch >> 4]);
+                        builder.append(HEXI_DECIMAL[ch & 15]);
+                    }
+                }
+                builder.append(' ');
+                for (int j = 0; j < width; j++) {
+                    if (j == width / 2)
+                        builder.append(' ');
+                    if (i + j < offset || i + j >= offset + len) {
+                        builder.append(' ');
+
+                    } else {
+                        int ch = bytes.readUnsignedByte(i + j);
+                        if (ch < ' ' || ch > 126)
+                            ch = '\u00B7';
+                        builder.append((char) ch);
+                    }
+                }
+                builder.append("\n");
+            }
+            return builder.toString();
+        } finally {
+            bytes.readLimit(limit);
+            bytes.readPosition(position);
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[2/7] cassandra git commit: Add fqltool replay

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/src/java/org/apache/cassandra/tools/fqltool/commands/Replay.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/fqltool/commands/Replay.java b/src/java/org/apache/cassandra/tools/fqltool/commands/Replay.java
new file mode 100644
index 0000000..043ead8
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/fqltool/commands/Replay.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.tools.fqltool.commands;
+
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import io.airlift.airline.Option;
+import net.openhft.chronicle.core.io.Closeable;
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+
+import org.apache.cassandra.tools.fqltool.FQLQuery;
+import org.apache.cassandra.tools.fqltool.FQLQueryIterator;
+import org.apache.cassandra.tools.fqltool.QueryReplayer;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.MergeIterator;
+
+/**
+ * replay the contents of a list of paths containing full query logs
+ */
+@Command(name = "replay", description = "Replay full query logs")
+public class Replay implements Runnable
+{
+    @Arguments(usage = "<path1> [<path2>...<pathN>]", description = "Paths containing the full query logs to replay.", required = true)
+    private List<String> arguments = new ArrayList<>();
+
+    @Option(title = "target", name = {"--target"}, description = "Hosts to replay the logs to, can be repeated to replay to more hosts.")
+    private List<String> targetHosts;
+
+    @Option(title = "results", name = { "--results"}, description = "Where to store the results of the queries, this should be a directory. Leave this option out to avoid storing results.")
+    private String resultPath;
+
+    @Option(title = "keyspace", name = { "--keyspace"}, description = "Only replay queries against this keyspace and queries without keyspace set.")
+    private String keyspace;
+
+    @Option(title = "debug", name = {"--debug"}, description = "Debug mode, print all queries executed.")
+    private boolean debug;
+
+    @Option(title = "store_queries", name = {"--store-queries"}, description = "Path to store the queries executed. Stores queries in the same order as the result sets are in the result files. Requires --results")
+    private String queryStorePath;
+
+    @Override
+    public void run()
+    {
+        try
+        {
+            List<File> resultPaths = null;
+            if (resultPath != null)
+            {
+                File basePath = new File(resultPath);
+                if (!basePath.exists() || !basePath.isDirectory())
+                {
+                    System.err.println("The results path (" + basePath + ") should be an existing directory");
+                    System.exit(1);
+                }
+                resultPaths = targetHosts.stream().map(target -> new File(basePath, target)).collect(Collectors.toList());
+                resultPaths.forEach(File::mkdir);
+            }
+            if (targetHosts.size() < 1)
+            {
+                System.err.println("You need to state at least one --target host to replay the query against");
+                System.exit(1);
+            }
+            replay(keyspace, arguments, targetHosts, resultPaths, queryStorePath, debug);
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static void replay(String keyspace, List<String> arguments, List<String> targetHosts, List<File> resultPaths, String queryStorePath, boolean debug)
+    {
+        int readAhead = 200; // how many fql queries should we read in to memory to be able to sort them?
+        List<ChronicleQueue> readQueues = null;
+        List<FQLQueryIterator> iterators = null;
+        List<Predicate<FQLQuery>> filters = new ArrayList<>();
+
+        if (keyspace != null)
+            filters.add(fqlQuery -> fqlQuery.keyspace == null || fqlQuery.keyspace.equals(keyspace));
+
+        try
+        {
+            readQueues = arguments.stream().map(s -> ChronicleQueueBuilder.single(s).readOnly(true).build()).collect(Collectors.toList());
+            iterators = readQueues.stream().map(ChronicleQueue::createTailer).map(tailer -> new FQLQueryIterator(tailer, readAhead)).collect(Collectors.toList());
+            try (MergeIterator<FQLQuery, List<FQLQuery>> iter = MergeIterator.get(iterators, FQLQuery::compareTo, new Reducer());
+                 QueryReplayer replayer = new QueryReplayer(iter, targetHosts, resultPaths, filters, System.out, queryStorePath, debug))
+            {
+                replayer.replay();
+            }
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+        finally
+        {
+            if (iterators != null)
+                iterators.forEach(AbstractIterator::close);
+            if (readQueues != null)
+                readQueues.forEach(Closeable::close);
+        }
+    }
+
+    @VisibleForTesting
+    public static class Reducer extends MergeIterator.Reducer<FQLQuery, List<FQLQuery>>
+    {
+        List<FQLQuery> queries = new ArrayList<>();
+        public void reduce(int idx, FQLQuery current)
+        {
+            queries.add(current);
+        }
+
+        protected List<FQLQuery> getReduced()
+        {
+            return queries;
+        }
+        protected void onKeyChange()
+        {
+            queries.clear();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62ffb772/test/unit/org/apache/cassandra/tools/fqltool/FQLReplayTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/fqltool/FQLReplayTest.java b/test/unit/org/apache/cassandra/tools/fqltool/FQLReplayTest.java
new file mode 100644
index 0000000..a662699
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/fqltool/FQLReplayTest.java
@@ -0,0 +1,760 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.fqltool;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import net.openhft.chronicle.queue.ChronicleQueue;
+import net.openhft.chronicle.queue.ChronicleQueueBuilder;
+import net.openhft.chronicle.queue.ExcerptAppender;
+import net.openhft.chronicle.queue.ExcerptTailer;
+import net.openhft.chronicle.wire.ValueIn;
+import org.apache.cassandra.audit.FullQueryLogger;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.tools.Util;
+import org.apache.cassandra.tools.fqltool.commands.Replay;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.MergeIterator;
+import org.apache.cassandra.utils.Pair;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class FQLReplayTest
+{
+    public FQLReplayTest()
+    {
+        Util.initDatabaseDescriptor();
+    }
+
+    @Test
+    public void testOrderedReplay() throws IOException
+    {
+        File f = generateQueries(100, true);
+        int queryCount = 0;
+        try (ChronicleQueue queue = ChronicleQueueBuilder.single(f).build();
+             FQLQueryIterator iter = new FQLQueryIterator(queue.createTailer(), 101))
+        {
+            long last = -1;
+            while (iter.hasNext())
+            {
+                FQLQuery q = iter.next();
+                assertTrue(q.queryStartTime >= last);
+                last = q.queryStartTime;
+                queryCount++;
+            }
+        }
+        assertEquals(100, queryCount);
+    }
+    @Test
+    public void testMergingIterator() throws IOException
+    {
+        File f = generateQueries(100, false);
+        File f2 = generateQueries(100, false);
+        int queryCount = 0;
+        try (ChronicleQueue queue = ChronicleQueueBuilder.single(f).build();
+             ChronicleQueue queue2 = ChronicleQueueBuilder.single(f2).build();
+             FQLQueryIterator iter = new FQLQueryIterator(queue.createTailer(), 101);
+             FQLQueryIterator iter2 = new FQLQueryIterator(queue2.createTailer(), 101);
+             MergeIterator<FQLQuery, List<FQLQuery>> merger = MergeIterator.get(Lists.newArrayList(iter, iter2), FQLQuery::compareTo, new Replay.Reducer()))
+        {
+            long last = -1;
+
+            while (merger.hasNext())
+            {
+                List<FQLQuery> qs = merger.next();
+                assertEquals(2, qs.size());
+                assertEquals(0, qs.get(0).compareTo(qs.get(1)));
+                assertTrue(qs.get(0).queryStartTime >= last);
+                last = qs.get(0).queryStartTime;
+                queryCount++;
+            }
+        }
+        assertEquals(100, queryCount);
+    }
+
+    @Test
+    public void testFQLQueryReader() throws IOException
+    {
+        FQLQueryReader reader = new FQLQueryReader();
+
+        try (ChronicleQueue queue = ChronicleQueueBuilder.single(generateQueries(1000, true)).build())
+        {
+            ExcerptTailer tailer = queue.createTailer();
+            int queryCount = 0;
+            while (tailer.readDocument(reader))
+            {
+                assertNotNull(reader.getQuery());
+                if (reader.getQuery() instanceof FQLQuery.Single)
+                {
+                    assertTrue(reader.getQuery().keyspace == null || reader.getQuery().keyspace.equals("querykeyspace"));
+                }
+                else
+                {
+                    assertEquals("someks", reader.getQuery().keyspace);
+                }
+                queryCount++;
+            }
+            assertEquals(1000, queryCount);
+        }
+    }
+
+    @Test
+    public void testStoringResults() throws Throwable
+    {
+        File tmpDir = Files.createTempDirectory("results").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, true);
+        ResultStore rs = new ResultStore(Collections.singletonList(tmpDir), queryDir);
+        try
+        {
+            FQLQuery query = new FQLQuery.Single("abc", 3, QueryOptions.DEFAULT, 12345, 11111, 22, "select * from abc", Collections.emptyList());
+            rs.storeColumnDefinitions(query, Collections.singletonList(res.getColumnDefinitions()));
+            Iterator<ResultHandler.ComparableRow> it = res.iterator();
+            while (it.hasNext())
+            {
+                List<ResultHandler.ComparableRow> row = Collections.singletonList(it.next());
+                rs.storeRows(row);
+            }
+            // this marks the end of the result set:
+            rs.storeRows(Collections.singletonList(null));
+        }
+        finally
+        {
+            rs.close();
+        }
+
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> resultSets = readResultFile(tmpDir, queryDir);
+        assertEquals(1, resultSets.size());
+        assertEquals(res, resultSets.get(0).right);
+
+    }
+
+    @Test
+    public void testCompareColumnDefinitions()
+    {
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultComparator rc = new ResultComparator();
+
+        List<ResultHandler.ComparableColumnDefinitions> colDefs = new ArrayList<>(100);
+        List<String> targetHosts = new ArrayList<>(100);
+        for (int i = 0; i < 100; i++)
+        {
+            targetHosts.add("host"+i);
+            colDefs.add(res.getColumnDefinitions());
+        }
+        assertTrue(rc.compareColumnDefinitions(targetHosts, null, colDefs));
+        colDefs.set(50, createResultSet(9, 9, false).getColumnDefinitions());
+        assertFalse(rc.compareColumnDefinitions(targetHosts, null, colDefs));
+    }
+
+    @Test
+    public void testCompareEqualRows()
+    {
+        ResultComparator rc = new ResultComparator();
+
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2);
+        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
+
+        while (true)
+        {
+            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
+            assertTrue(rc.compareRows(Lists.newArrayList("eq1", "eq2"), null, rows));
+            if (rows.stream().allMatch(Objects::isNull))
+                break;
+        }
+    }
+
+    @Test
+    public void testCompareRowsDifferentCount()
+    {
+        ResultComparator rc = new ResultComparator();
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(10, 11, false));
+        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
+        boolean foundMismatch = false;
+        while (true)
+        {
+            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
+            if (rows.stream().allMatch(Objects::isNull))
+                break;
+            if (!rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows))
+            {
+                foundMismatch = true;
+            }
+        }
+        assertTrue(foundMismatch);
+    }
+
+    @Test
+    public void testCompareRowsDifferentContent()
+    {
+        ResultComparator rc = new ResultComparator();
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(10, 10, true));
+        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
+        while (true)
+        {
+            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
+            if (rows.stream().allMatch(Objects::isNull))
+                break;
+            assertFalse(rows.toString(), rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows));
+        }
+    }
+
+    @Test
+    public void testCompareRowsDifferentColumnCount()
+    {
+        ResultComparator rc = new ResultComparator();
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, createResultSet(11, 10, false));
+        List<Iterator<ResultHandler.ComparableRow>> iters = toCompare.stream().map(Iterable::iterator).collect(Collectors.toList());
+        while (true)
+        {
+            List<ResultHandler.ComparableRow> rows = ResultHandler.rows(iters);
+            if (rows.stream().allMatch(Objects::isNull))
+                break;
+            assertFalse(rows.toString(), rc.compareRows(Lists.newArrayList("eq1", "eq2", "diff"), null, rows));
+        }
+    }
+
+    @Test
+    public void testResultHandler() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
+        ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir);
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res3 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, res3);
+        rh.handleResults(new FQLQuery.Single("abcabc", 3, QueryOptions.DEFAULT, 1111, 2222, 3333, "select * from xyz", Collections.emptyList()), toCompare);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(resultPaths.get(0), queryDir);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results2 = readResultFile(resultPaths.get(1), queryDir);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results3 = readResultFile(resultPaths.get(2), queryDir);
+        assertEquals(results1, results2);
+        assertEquals(results1, results3);
+        assertEquals(Iterables.getOnlyElement(results3).right, res);
+    }
+
+    @Test
+    public void testResultHandlerWithDifference() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
+        ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir);
+        ResultHandler.ComparableResultSet res = createResultSet(10, 10, false);
+        ResultHandler.ComparableResultSet res2 = createResultSet(10, 5, false);
+        ResultHandler.ComparableResultSet res3 = createResultSet(10, 10, false);
+        List<ResultHandler.ComparableResultSet> toCompare = Lists.newArrayList(res, res2, res3);
+        rh.handleResults(new FQLQuery.Single("aaa", 3, QueryOptions.DEFAULT, 123123, 11111, 22222, "select * from abcabc", Collections.emptyList()), toCompare);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(resultPaths.get(0), queryDir);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results2 = readResultFile(resultPaths.get(1), queryDir);
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results3 = readResultFile(resultPaths.get(2), queryDir);
+        assertEquals(results1, results3);
+        assertEquals(results2.get(0).right, res2);
+    }
+
+    @Test
+    public void testResultHandlerMultipleResultSets() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
+        ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir);
+        List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets = new ArrayList<>();
+        Random random = new Random();
+        for (int i = 0; i < 10; i++)
+        {
+            List<ResultHandler.ComparableResultSet> results = new ArrayList<>();
+            List<ByteBuffer> values = Collections.singletonList(ByteBufferUtil.bytes(i * 50));
+            for (int jj = 0; jj < targetHosts.size(); jj++)
+            {
+                results.add(createResultSet(5, 1 + random.nextInt(10), true));
+            }
+            FQLQuery q = new FQLQuery.Single("abc"+i,
+                                             3,
+                                             QueryOptions.forInternalCalls(values),
+                                             i * 1000,
+                                             12345,
+                                             54321,
+                                             "select * from xyz where id = "+i,
+                                             values);
+            resultSets.add(Pair.create(q, results));
+        }
+        for (int i = 0; i < resultSets.size(); i++)
+            rh.handleResults(resultSets.get(i).left, resultSets.get(i).right);
+
+        for (int i = 0; i < targetHosts.size(); i++)
+            compareWithFile(resultPaths, queryDir, resultSets, i);
+    }
+
+    @Test
+    public void testResultHandlerFailedQuery() throws IOException
+    {
+        List<String> targetHosts = Lists.newArrayList("hosta", "hostb", "hostc", "hostd");
+        File tmpDir = Files.createTempDirectory("testresulthandler").toFile();
+        File queryDir = Files.createTempDirectory("queries").toFile();
+        List<File> resultPaths = new ArrayList<>();
+        targetHosts.forEach(host -> { File f = new File(tmpDir, host); f.mkdir(); resultPaths.add(f);});
+        ResultHandler rh = new ResultHandler(targetHosts, resultPaths, queryDir);
+        List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets = new ArrayList<>();
+        Random random = new Random();
+        for (int i = 0; i < 10; i++)
+        {
+            List<ResultHandler.ComparableResultSet> results = new ArrayList<>();
+            List<ByteBuffer> values = Collections.singletonList(ByteBufferUtil.bytes(i * 50));
+            for (int jj = 0; jj < targetHosts.size(); jj++)
+            {
+                results.add(createResultSet(5, 1 + random.nextInt(10), true));
+            }
+            results.set(0, FakeResultSet.failed(new RuntimeException("testing abc")));
+            results.set(3, FakeResultSet.failed(new RuntimeException("testing abc")));
+            FQLQuery q = new FQLQuery.Single("abc"+i,
+                                             3,
+                                             QueryOptions.forInternalCalls(values),
+                                             i * 1000,
+                                             i * 12345,
+                                             i * 54321,
+                                             "select * from xyz where id = "+i,
+                                             values);
+            resultSets.add(Pair.create(q, results));
+        }
+        for (int i = 0; i < resultSets.size(); i++)
+            rh.handleResults(resultSets.get(i).left, resultSets.get(i).right);
+
+        for (int i = 0; i < targetHosts.size(); i++)
+            compareWithFile(resultPaths, queryDir, resultSets, i);
+    }
+
+
+    @Test
+    public void testCompare()
+    {
+        FQLQuery q1 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.emptyList());
+        FQLQuery q2 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222,"aaaa", Collections.emptyList());
+
+        assertEquals(0, q1.compareTo(q2));
+        assertEquals(0, q2.compareTo(q1));
+
+        FQLQuery q3 = new FQLQuery.Batch("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, com.datastax.driver.core.BatchStatement.Type.UNLOGGED, Collections.emptyList(), Collections.emptyList());
+        // single queries before batch queries
+        assertTrue(q1.compareTo(q3) < 0);
+        assertTrue(q3.compareTo(q1) > 0);
+
+        // check that smaller query time
+        FQLQuery q4 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 124, 111, 222, "aaaa", Collections.emptyList());
+        assertTrue(q1.compareTo(q4) < 0);
+        assertTrue(q4.compareTo(q1) > 0);
+
+        FQLQuery q5 = new FQLQuery.Batch("abc", 0, QueryOptions.DEFAULT, 124, 111, 222, com.datastax.driver.core.BatchStatement.Type.UNLOGGED, Collections.emptyList(), Collections.emptyList());
+        assertTrue(q1.compareTo(q5) < 0);
+        assertTrue(q5.compareTo(q1) > 0);
+
+        FQLQuery q6 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes(10)));
+        FQLQuery q7 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.emptyList());
+        assertTrue(q6.compareTo(q7) > 0);
+        assertTrue(q7.compareTo(q6) < 0);
+
+        FQLQuery q8 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes("a")));
+        FQLQuery q9 = new FQLQuery.Single("abc", 0, QueryOptions.DEFAULT, 123, 111, 222, "aaaa", Collections.singletonList(ByteBufferUtil.bytes("b")));
+        assertTrue(q8.compareTo(q9) < 0);
+        assertTrue(q9.compareTo(q8) > 0);
+    }
+
+    private File generateQueries(int count, boolean random) throws IOException
+    {
+        Random r = new Random();
+        File dir = Files.createTempDirectory("chronicle").toFile();
+        try (ChronicleQueue readQueue = ChronicleQueueBuilder.single(dir).build())
+        {
+            ExcerptAppender appender = readQueue.acquireAppender();
+
+            for (int i = 0; i < count; i++)
+            {
+                long timestamp = random ? Math.abs(r.nextLong() % 10000) : i;
+                if (random ? r.nextBoolean() : i % 2 == 0)
+                {
+                    String query = "abcdefghijklm " + i;
+                    QueryState qs = r.nextBoolean() ? queryState() : queryState("querykeyspace");
+                    FullQueryLogger.Query  q = new FullQueryLogger.Query(query, QueryOptions.DEFAULT, qs, timestamp);
+                    appender.writeDocument(q);
+                    q.release();
+                }
+                else
+                {
+                    int batchSize = random ? r.nextInt(99) + 1 : i + 1;
+                    List<String> queries = new ArrayList<>(batchSize);
+                    List<List<ByteBuffer>> values = new ArrayList<>(batchSize);
+                    for (int jj = 0; jj < (random ? r.nextInt(batchSize) : 10); jj++)
+                    {
+                        queries.add("aaaaaa batch "+i+":"+jj);
+                        values.add(Collections.emptyList());
+                    }
+                    FullQueryLogger.Batch batch = new FullQueryLogger.Batch(BatchStatement.Type.UNLOGGED,
+                                                                            queries,
+                                                                            values,
+                                                                            QueryOptions.DEFAULT,
+                                                                            queryState("someks"),
+                                                                            timestamp);
+                    appender.writeDocument(batch);
+                    batch.release();
+                }
+            }
+        }
+        return dir;
+    }
+
+    private QueryState queryState()
+    {
+        return QueryState.forInternalCalls();
+    }
+
+    private QueryState queryState(String keyspace)
+    {
+        ClientState clientState = ClientState.forInternalCalls(keyspace);
+        return new QueryState(clientState);
+    }
+
+    private static ResultHandler.ComparableResultSet createResultSet(int columnCount, int rowCount, boolean random)
+    {
+        List<Pair<String, String>> columnDefs = new ArrayList<>(columnCount);
+        Random r = new Random();
+        for (int i = 0; i < columnCount; i++)
+        {
+            columnDefs.add(Pair.create("a" + i, "int"));
+        }
+        List<List<String>> rows = new ArrayList<>();
+        for (int i = 0; i < rowCount; i++)
+        {
+            List<String> row = new ArrayList<>(columnCount);
+            for (int jj = 0; jj < columnCount; jj++)
+                row.add(i + " col " + jj + (random ? r.nextInt() : ""));
+            rows.add(row);
+        }
+        return new FakeResultSet(columnDefs, rows);
+    }
+
+    private static void compareWithFile(List<File> dirs, File resultDir, List<Pair<FQLQuery, List<ResultHandler.ComparableResultSet>>> resultSets, int idx)
+    {
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> results1 = readResultFile(dirs.get(idx), resultDir);
+        for (int i = 0; i < results1.size(); i++)
+        {
+            assertEquals(results1.get(i).left, resultSets.get(i).left);
+            assertEquals(results1.get(i).right, resultSets.get(i).right.get(idx));
+        }
+    }
+
+    private static List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> readResultFile(File dir, File queryDir)
+    {
+        List<Pair<FQLQuery, ResultHandler.ComparableResultSet>> resultSets = new ArrayList<>();
+        try (ChronicleQueue q = ChronicleQueueBuilder.single(dir).build();
+             ChronicleQueue queryQ = ChronicleQueueBuilder.single(queryDir).build())
+        {
+            ExcerptTailer tailer = q.createTailer();
+            ExcerptTailer queryTailer = queryQ.createTailer();
+            List<Pair<String, String>> columnDefinitions = new ArrayList<>();
+            List<List<String>> rowColumns = new ArrayList<>();
+            AtomicBoolean allRowsRead = new AtomicBoolean(false);
+            AtomicBoolean failedQuery = new AtomicBoolean(false);
+            while (tailer.readDocument(wire -> {
+                String type = wire.read("type").text();
+                if (type.equals("column_definitions"))
+                {
+                    int columnCount = wire.read("column_count").int32();
+                    for (int i = 0; i < columnCount; i++)
+                    {
+                        ValueIn vi = wire.read("column_definition");
+                        String name = vi.text();
+                        String dataType = vi.text();
+                        columnDefinitions.add(Pair.create(name, dataType));
+                    }
+                }
+                else if (type.equals("row"))
+                {
+                    int rowColumnCount = wire.read("row_column_count").int32();
+                    List<String> r = new ArrayList<>(rowColumnCount);
+                    for (int i = 0; i < rowColumnCount; i++)
+                    {
+                        byte[] b = wire.read("column").bytes();
+                        r.add(new String(b));
+                    }
+                    rowColumns.add(r);
+                }
+                else if (type.equals("end_resultset"))
+                {
+                    allRowsRead.set(true);
+                }
+                else if (type.equals("query_failed"))
+                {
+                    failedQuery.set(true);
+                }
+            }))
+            {
+                if (allRowsRead.get())
+                {
+                    FQLQueryReader reader = new FQLQueryReader();
+                    queryTailer.readDocument(reader);
+                    resultSets.add(Pair.create(reader.getQuery(), failedQuery.get() ? FakeResultSet.failed(new RuntimeException("failure"))
+                                                                                    : new FakeResultSet(ImmutableList.copyOf(columnDefinitions), ImmutableList.copyOf(rowColumns))));
+                    allRowsRead.set(false);
+                    failedQuery.set(false);
+                    columnDefinitions.clear();
+                    rowColumns.clear();
+                }
+            }
+        }
+        return resultSets;
+    }
+
+    private static class FakeResultSet implements ResultHandler.ComparableResultSet
+    {
+        private final List<Pair<String, String>> cdStrings;
+        private final List<List<String>> rows;
+        private final Throwable ex;
+
+        public FakeResultSet(List<Pair<String, String>> cdStrings, List<List<String>> rows)
+        {
+            this(cdStrings, rows, null);
+        }
+
+        public FakeResultSet(List<Pair<String, String>> cdStrings, List<List<String>> rows, Throwable ex)
+        {
+            this.cdStrings = cdStrings;
+            this.rows = rows;
+            this.ex = ex;
+        }
+
+        public static FakeResultSet failed(Throwable ex)
+        {
+            return new FakeResultSet(null, null, ex);
+        }
+
+        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+        {
+            return new FakeComparableColumnDefinitions(cdStrings, wasFailed());
+        }
+
+        public boolean wasFailed()
+        {
+            return getFailureException() != null;
+        }
+
+        public Throwable getFailureException()
+        {
+            return ex;
+        }
+
+        public Iterator<ResultHandler.ComparableRow> iterator()
+        {
+            if (wasFailed())
+                return Collections.emptyListIterator();
+            return new AbstractIterator<ResultHandler.ComparableRow>()
+            {
+                Iterator<List<String>> iter = rows.iterator();
+                protected ResultHandler.ComparableRow computeNext()
+                {
+                    if (iter.hasNext())
+                        return new FakeComparableRow(iter.next(), cdStrings);
+                    return endOfData();
+                }
+            };
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (!(o instanceof FakeResultSet)) return false;
+            FakeResultSet that = (FakeResultSet) o;
+            if (wasFailed() && that.wasFailed())
+                return true;
+            return Objects.equals(cdStrings, that.cdStrings) &&
+                   Objects.equals(rows, that.rows);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(cdStrings, rows);
+        }
+
+        public String toString()
+        {
+            return "FakeResultSet{" +
+                   "cdStrings=" + cdStrings +
+                   ", rows=" + rows +
+                   '}';
+        }
+    }
+
+    private static class FakeComparableRow implements ResultHandler.ComparableRow
+    {
+        private final List<String> row;
+        private final List<Pair<String, String>> cds;
+
+        public FakeComparableRow(List<String> row, List<Pair<String,String>> cds)
+        {
+            this.row = row;
+            this.cds = cds;
+        }
+
+        public ByteBuffer getBytesUnsafe(int i)
+        {
+            return ByteBufferUtil.bytes(row.get(i));
+        }
+
+        public ResultHandler.ComparableColumnDefinitions getColumnDefinitions()
+        {
+            return new FakeComparableColumnDefinitions(cds, false);
+        }
+
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof FakeComparableRow))
+                return false;
+            return row.equals(((FakeComparableRow)other).row);
+        }
+
+        public String toString()
+        {
+            return row.toString();
+        }
+    }
+
+    private static class FakeComparableColumnDefinitions implements ResultHandler.ComparableColumnDefinitions
+    {
+        private final List<ResultHandler.ComparableDefinition> defs;
+        private final boolean failed;
+        public FakeComparableColumnDefinitions(List<Pair<String, String>> cds, boolean failed)
+        {
+            defs = cds != null ? cds.stream().map(FakeComparableDefinition::new).collect(Collectors.toList()) : null;
+            this.failed = failed;
+        }
+
+        public List<ResultHandler.ComparableDefinition> asList()
+        {
+            if (wasFailed())
+                return Collections.emptyList();
+            return defs;
+        }
+
+        public boolean wasFailed()
+        {
+            return failed;
+        }
+
+        public int size()
+        {
+            return defs.size();
+        }
+
+        public Iterator<ResultHandler.ComparableDefinition> iterator()
+        {
+            if (wasFailed())
+                return Collections.emptyListIterator();
+            return new AbstractIterator<ResultHandler.ComparableDefinition>()
+            {
+                Iterator<ResultHandler.ComparableDefinition> iter = defs.iterator();
+                protected ResultHandler.ComparableDefinition computeNext()
+                {
+                    if (iter.hasNext())
+                        return iter.next();
+                    return endOfData();
+                }
+            };
+        }
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof FakeComparableColumnDefinitions))
+                return false;
+            return defs.equals(((FakeComparableColumnDefinitions)other).defs);
+        }
+
+        public String toString()
+        {
+            return defs.toString();
+        }
+    }
+
+    private static class FakeComparableDefinition implements ResultHandler.ComparableDefinition
+    {
+        private final Pair<String, String> p;
+
+        public FakeComparableDefinition(Pair<String, String> p)
+        {
+            this.p = p;
+        }
+        public String getType()
+        {
+            return p.right;
+        }
+
+        public String getName()
+        {
+            return p.left;
+        }
+
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof FakeComparableDefinition))
+                return false;
+            return p.equals(((FakeComparableDefinition)other).p);
+        }
+
+        public String toString()
+        {
+            return getName() + ':' + getType();
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org