You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2016/12/13 09:27:27 UTC

[03/26] cassandra git commit: Thrift removal

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 23e6df3..0c55f76 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -76,7 +76,6 @@ public class NodeTool
                 EnableGossip.class,
                 DisableGossip.class,
                 EnableHandoff.class,
-                EnableThrift.class,
                 GcStats.class,
                 GetCompactionThreshold.class,
                 GetCompactionThroughput.class,
@@ -116,7 +115,6 @@ public class NodeTool
                 Status.class,
                 StatusBinary.class,
                 StatusGossip.class,
-                StatusThrift.class,
                 StatusBackup.class,
                 StatusHandoff.class,
                 Stop.class,
@@ -130,7 +128,6 @@ public class NodeTool
                 ResetLocalSchema.class,
                 ReloadTriggers.class,
                 SetCacheKeysToSave.class,
-                DisableThrift.class,
                 DisableHandoff.class,
                 Drain.class,
                 TruncateHints.class,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/src/java/org/apache/cassandra/tools/nodetool/DisableThrift.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableThrift.java b/src/java/org/apache/cassandra/tools/nodetool/DisableThrift.java
deleted file mode 100644
index 148b195..0000000
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableThrift.java
+++ /dev/null
@@ -1,33 +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.nodetool;
-
-import io.airlift.command.Command;
-
-import org.apache.cassandra.tools.NodeProbe;
-import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
-
-@Command(name = "disablethrift", description = "Disable thrift server")
-public class DisableThrift extends NodeToolCmd
-{
-    @Override
-    public void execute(NodeProbe probe)
-    {
-        probe.stopThriftServer();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/src/java/org/apache/cassandra/tools/nodetool/EnableThrift.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableThrift.java b/src/java/org/apache/cassandra/tools/nodetool/EnableThrift.java
deleted file mode 100644
index 780b36d..0000000
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableThrift.java
+++ /dev/null
@@ -1,33 +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.nodetool;
-
-import io.airlift.command.Command;
-
-import org.apache.cassandra.tools.NodeProbe;
-import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
-
-@Command(name = "enablethrift", description = "Reenable thrift server")
-public class EnableThrift extends NodeToolCmd
-{
-    @Override
-    public void execute(NodeProbe probe)
-    {
-        probe.startThriftServer();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/src/java/org/apache/cassandra/tools/nodetool/Info.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Info.java b/src/java/org/apache/cassandra/tools/nodetool/Info.java
index 032e47f..bddd124 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Info.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Info.java
@@ -47,7 +47,6 @@ public class Info extends NodeToolCmd
 
         System.out.printf("%-23s: %s%n", "ID", probe.getLocalHostId());
         System.out.printf("%-23s: %s%n", "Gossip active", gossipInitialized);
-        System.out.printf("%-23s: %s%n", "Thrift active", probe.isThriftServerRunning());
         System.out.printf("%-23s: %s%n", "Native Transport active", probe.isNativeTransportRunning());
         System.out.printf("%-23s: %s%n", "Load", probe.getLoadString());
         if (gossipInitialized)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/src/java/org/apache/cassandra/tools/nodetool/StatusThrift.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/StatusThrift.java b/src/java/org/apache/cassandra/tools/nodetool/StatusThrift.java
deleted file mode 100644
index 0cb17d2..0000000
--- a/src/java/org/apache/cassandra/tools/nodetool/StatusThrift.java
+++ /dev/null
@@ -1,36 +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.nodetool;
-
-import io.airlift.command.Command;
-
-import org.apache.cassandra.tools.NodeProbe;
-import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
-
-@Command(name = "statusthrift", description = "Status of thrift server")
-public class StatusThrift extends NodeToolCmd
-{
-    @Override
-    public void execute(NodeProbe probe)
-    {
-        System.out.println(
-                probe.isThriftServerRunning()
-                ? "running"
-                : "not running");
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/ResultMessage.java b/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
index 05a1276..334aa89 100644
--- a/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
@@ -27,9 +27,6 @@ import org.apache.cassandra.cql3.ResultSet;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.transport.*;
-import org.apache.cassandra.thrift.CqlPreparedResult;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.thrift.CqlResultType;
 import org.apache.cassandra.utils.MD5Digest;
 
 public abstract class ResultMessage extends Message.Response
@@ -103,8 +100,6 @@ public abstract class ResultMessage extends Message.Response
         this.kind = kind;
     }
 
-    public abstract CqlResult toThriftResult();
-
     public static class Void extends ResultMessage
     {
         // Even though we have no specific information here, don't make a
@@ -132,11 +127,6 @@ public abstract class ResultMessage extends Message.Response
             }
         };
 
-        public CqlResult toThriftResult()
-        {
-            return new CqlResult(CqlResultType.VOID);
-        }
-
         @Override
         public String toString()
         {
@@ -175,11 +165,6 @@ public abstract class ResultMessage extends Message.Response
             }
         };
 
-        public CqlResult toThriftResult()
-        {
-            return new CqlResult(CqlResultType.VOID);
-        }
-
         @Override
         public String toString()
         {
@@ -219,11 +204,6 @@ public abstract class ResultMessage extends Message.Response
             this.result = result;
         }
 
-        public CqlResult toThriftResult()
-        {
-            return result.toThriftResult();
-        }
-
         @Override
         public String toString()
         {
@@ -244,7 +224,7 @@ public abstract class ResultMessage extends Message.Response
                 if (version.isGreaterThan(ProtocolVersion.V1))
                     resultMetadata = ResultSet.ResultMetadata.codec.decode(body, version);
 
-                return new Prepared(id, -1, metadata, resultMetadata);
+                return new Prepared(id, metadata, resultMetadata);
             }
 
             public void encode(ResultMessage msg, ByteBuf dest, ProtocolVersion version)
@@ -282,24 +262,15 @@ public abstract class ResultMessage extends Message.Response
         /** Describes the results of executing this prepared statement */
         public final ResultSet.ResultMetadata resultMetadata;
 
-        // statement id for CQL-over-thrift compatibility. The binary protocol ignore that.
-        private final int thriftStatementId;
-
         public Prepared(MD5Digest statementId, ParsedStatement.Prepared prepared)
         {
-            this(statementId, -1, new ResultSet.PreparedMetadata(prepared.boundNames, prepared.partitionKeyBindIndexes), extractResultMetadata(prepared.statement));
+            this(statementId, new ResultSet.PreparedMetadata(prepared.boundNames, prepared.partitionKeyBindIndexes), extractResultMetadata(prepared.statement));
         }
 
-        public static Prepared forThrift(int statementId, List<ColumnSpecification> names)
-        {
-            return new Prepared(null, statementId, new ResultSet.PreparedMetadata(names, null), ResultSet.ResultMetadata.EMPTY);
-        }
-
-        private Prepared(MD5Digest statementId, int thriftStatementId, ResultSet.PreparedMetadata metadata, ResultSet.ResultMetadata resultMetadata)
+        private Prepared(MD5Digest statementId, ResultSet.PreparedMetadata metadata, ResultSet.ResultMetadata resultMetadata)
         {
             super(Kind.PREPARED);
             this.statementId = statementId;
-            this.thriftStatementId = thriftStatementId;
             this.metadata = metadata;
             this.resultMetadata = resultMetadata;
         }
@@ -312,23 +283,6 @@ public abstract class ResultMessage extends Message.Response
             return ((SelectStatement)statement).getResultMetadata();
         }
 
-        public CqlResult toThriftResult()
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public CqlPreparedResult toThriftPreparedResult()
-        {
-            List<String> namesString = new ArrayList<String>(metadata.names.size());
-            List<String> typesString = new ArrayList<String>(metadata.names.size());
-            for (ColumnSpecification name : metadata.names)
-            {
-                namesString.add(name.toString());
-                typesString.add(name.type.toString());
-            }
-            return new CqlPreparedResult(thriftStatementId, metadata.names.size()).setVariable_types(typesString).setVariable_names(namesString);
-        }
-
         @Override
         public String toString()
         {
@@ -368,11 +322,6 @@ public abstract class ResultMessage extends Message.Response
             }
         };
 
-        public CqlResult toThriftResult()
-        {
-            return new CqlResult(CqlResultType.VOID);
-        }
-
         @Override
         public String toString()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/src/java/org/apache/cassandra/utils/BatchRemoveIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/BatchRemoveIterator.java b/src/java/org/apache/cassandra/utils/BatchRemoveIterator.java
deleted file mode 100644
index 4377426..0000000
--- a/src/java/org/apache/cassandra/utils/BatchRemoveIterator.java
+++ /dev/null
@@ -1,32 +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.utils;
-
-import java.util.Iterator;
-
-/**
- * Iterator that allows us to more efficiently remove many items
- */
-public interface BatchRemoveIterator<T> extends Iterator<T>
-{
-    /**
-     * Commits the remove operations in this batch iterator. After this no more
-     * deletes can be made. Any further calls to remove() or commit() will throw IllegalStateException.
-     */
-    void commit();
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
index b6f93bc..151eb18 100644
--- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
+++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
@@ -411,6 +411,15 @@ public class ByteBufferUtil
         return bytes;
     }
 
+    public static byte[] readBytesWithLength(DataInput in) throws IOException
+    {
+        int length = in.readInt();
+        if (length < 0)
+            throw new IOException("Corrupt (negative) value length encountered");
+
+        return readBytes(in, length);
+    }
+
     /**
      * Convert a byte buffer to an integer.
      * Does not change the byte buffer position.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/conf/cassandra-murmur.yaml
----------------------------------------------------------------------
diff --git a/test/conf/cassandra-murmur.yaml b/test/conf/cassandra-murmur.yaml
index a4b25ba..a8288d0 100644
--- a/test/conf/cassandra-murmur.yaml
+++ b/test/conf/cassandra-murmur.yaml
@@ -14,7 +14,6 @@ hints_directory: build/test/cassandra/hints
 partitioner: org.apache.cassandra.dht.Murmur3Partitioner
 listen_address: 127.0.0.1
 storage_port: 7010
-rpc_port: 9170
 start_native_transport: true
 native_transport_port: 9042
 column_index_size_in_kb: 4
@@ -28,8 +27,6 @@ seed_provider:
           - seeds: "127.0.0.1"
 endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
 dynamic_snitch: true
-request_scheduler: org.apache.cassandra.scheduler.RoundRobinScheduler
-request_scheduler_id: keyspace
 server_encryption_options:
     internode_encryption: none
     keystore: conf/.keystore

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml
index cf02634..6a5d6e0 100644
--- a/test/conf/cassandra.yaml
+++ b/test/conf/cassandra.yaml
@@ -15,7 +15,6 @@ hints_directory: build/test/cassandra/hints
 partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
 listen_address: 127.0.0.1
 storage_port: 7010
-rpc_port: 9170
 start_native_transport: true
 native_transport_port: 9042
 column_index_size_in_kb: 4
@@ -29,8 +28,6 @@ seed_provider:
           - seeds: "127.0.0.1"
 endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
 dynamic_snitch: true
-request_scheduler: org.apache.cassandra.scheduler.RoundRobinScheduler
-request_scheduler_id: keyspace
 server_encryption_options:
     internode_encryption: none
     keystore: conf/.keystore

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/resources/functions/configure_cassandra.sh
----------------------------------------------------------------------
diff --git a/test/resources/functions/configure_cassandra.sh b/test/resources/functions/configure_cassandra.sh
index 3464653..38ff098 100644
--- a/test/resources/functions/configure_cassandra.sh
+++ b/test/resources/functions/configure_cassandra.sh
@@ -45,40 +45,25 @@ function configure_cassandra() {
       ;;
   esac
   
-  OH_SIX_CONFIG="/etc/cassandra/conf/storage-conf.xml"
-  
-  if [[ -e "$OH_SIX_CONFIG" ]] ; then 
-    config_file=$OH_SIX_CONFIG
-    seeds=""
+  config_file="/etc/cassandra/conf/cassandra.yaml"
+  if [[ "x"`grep -e '^seeds:' $config_file` == "x" ]]; then
+    seeds="$1" # 08 format seeds
+    shift
     for server in "$@"; do
-      seeds="${seeds}<Seed>${server}</Seed>"
+      seeds="${seeds},${server}"
     done
-  
-    #TODO set replication
-    sed -i -e "s|<Seed>127.0.0.1</Seed>|$seeds|" $config_file
-    sed -i -e "s|<ListenAddress>localhost</ListenAddress>|<ListenAddress>$PRIVATE_SELF_HOST</ListenAddress>|" $config_file
-    sed -i -e "s|<ThriftAddress>localhost</ThriftAddress>|<ThriftAddress>$PUBLIC_SELF_HOST</ThriftAddress>|" $config_file
+    sed -i -e "s|- seeds: \"127.0.0.1\"|- seeds: \"${seeds}\"|" $config_file
   else
-    config_file="/etc/cassandra/conf/cassandra.yaml"
-    if [[ "x"`grep -e '^seeds:' $config_file` == "x" ]]; then
-      seeds="$1" # 08 format seeds
-      shift
-      for server in "$@"; do
-        seeds="${seeds},${server}"
-      done
-      sed -i -e "s|- seeds: \"127.0.0.1\"|- seeds: \"${seeds}\"|" $config_file
-    else
-      seeds="" # 07 format seeds
-      for server in "$@"; do
-        seeds="${seeds}\n    - ${server}"
-      done
-      sed -i -e "/^seeds:/,/^/d" $config_file ; echo -e "seeds:${seeds}" >> $config_file
-    fi
-  
-    sed -i -e "s|listen_address: localhost|listen_address: $PRIVATE_SELF_HOST|" $config_file
-    sed -i -e "s|rpc_address: localhost|rpc_address: $PUBLIC_SELF_HOST|" $config_file
+    seeds="" # 07 format seeds
+    for server in "$@"; do
+      seeds="${seeds}\n    - ${server}"
+    done
+    sed -i -e "/^seeds:/,/^/d" $config_file ; echo -e "seeds:${seeds}" >> $config_file
   fi
   
+  sed -i -e "s|listen_address: localhost|listen_address: $PRIVATE_SELF_HOST|" $config_file
+  sed -i -e "s|rpc_address: localhost|rpc_address: $PUBLIC_SELF_HOST|" $config_file
+  
   # Now that it's configured, start Cassandra
   nohup /etc/rc.local &
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java b/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java
index 0047f48..0e4a3cf 100644
--- a/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java
+++ b/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java
@@ -49,7 +49,6 @@ public class OffsetAwareConfigurationLoader extends YamlConfigurationLoader
 
         String sep = File.pathSeparator;
 
-        config.rpc_port += offset;
         config.native_transport_port += offset;
         config.storage_port += offset;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 3fa24d7..d77ca78 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -680,7 +680,6 @@ public class Util
         {
             super(original.isDigestQuery(),
                   original.digestVersion(),
-                  original.isForThrift(),
                   original.metadata(),
                   original.nowInSec(),
                   original.columnFilter(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/client/TestRingCache.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/client/TestRingCache.java b/test/unit/org/apache/cassandra/client/TestRingCache.java
deleted file mode 100644
index 51bf566..0000000
--- a/test/unit/org/apache/cassandra/client/TestRingCache.java
+++ /dev/null
@@ -1,121 +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.client;
-
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.hadoop.ConfigHelper;
-import org.apache.cassandra.thrift.Cassandra;
-import org.apache.cassandra.thrift.Column;
-import org.apache.cassandra.thrift.ColumnParent;
-import org.apache.cassandra.thrift.ColumnPath;
-import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TSocket;
-
-
-/**
- *  Sample code that uses RingCache in the client.
- */
-public class TestRingCache
-{
-    private RingCache ringCache;
-    private Cassandra.Client thriftClient;
-    private Configuration conf;
-
-    public TestRingCache(String keyspace)
-    {
-        ConfigHelper.setOutputColumnFamily(conf, keyspace, "Standard1");
-    	ringCache = new RingCache(conf);
-    }
-
-    private void setup(String server, int port) throws Exception
-    {
-        /* Establish a thrift connection to the cassandra instance */
-        TSocket socket = new TSocket(server, port);
-        System.out.println(" connected to " + server + ":" + port + ".");
-        TBinaryProtocol binaryProtocol = new TBinaryProtocol(new TFramedTransport(socket));
-        Cassandra.Client cassandraClient = new Cassandra.Client(binaryProtocol);
-        socket.open();
-        thriftClient = cassandraClient;
-        String seed = DatabaseDescriptor.getSeeds().iterator().next().getHostAddress();
-        conf = new Configuration();
-        ConfigHelper.setOutputPartitioner(conf, DatabaseDescriptor.getPartitioner().getClass().getName());
-        ConfigHelper.setOutputInitialAddress(conf, seed);
-        ConfigHelper.setOutputRpcPort(conf, Integer.toString(DatabaseDescriptor.getRpcPort()));
-
-    }
-
-    /**
-     * usage: java -cp <configpath> org.apache.cassandra.client.TestRingCache [keyspace row-id-prefix row-id-int]
-     * to test a single keyspace/row, use the parameters. row-id-prefix and row-id-int are appended together to form a
-     * single row id.  If you supply now parameters, 'Keyspace1' is assumed and will check 9 rows ('row1' through 'row9').
-     * @param args
-     * @throws Exception
-     */
-    public static void main(String[] args) throws Throwable
-    {
-        int minRow;
-        int maxRow;
-        String rowPrefix, keyspace = "Keyspace1";
-
-        if (args.length > 0)
-        {
-            keyspace = args[0];
-            rowPrefix = args[1];
-            minRow = Integer.parseInt(args[2]);
-            maxRow = minRow + 1;
-        }
-        else
-        {
-            minRow = 1;
-            maxRow = 10;
-            rowPrefix = "row";
-        }
-
-        TestRingCache tester = new TestRingCache(keyspace);
-
-        for (int nRows = minRow; nRows < maxRow; nRows++)
-        {
-            ByteBuffer row = ByteBufferUtil.bytes((rowPrefix + nRows));
-            ColumnPath col = new ColumnPath("Standard1").setSuper_column((ByteBuffer)null).setColumn("col1".getBytes());
-            ColumnParent parent = new ColumnParent("Standard1").setSuper_column((ByteBuffer)null);
-
-            Collection<InetAddress> endpoints = tester.ringCache.getEndpoint(row);
-            InetAddress firstEndpoint = endpoints.iterator().next();
-            System.out.printf("hosts with key %s : %s; choose %s%n",
-                              new String(row.array()), StringUtils.join(endpoints, ","), firstEndpoint);
-
-            // now, read the row back directly from the host owning the row locally
-            tester.setup(firstEndpoint.getHostAddress(), DatabaseDescriptor.getRpcPort());
-            tester.thriftClient.set_keyspace(keyspace);
-            tester.thriftClient.insert(row, parent, new Column(ByteBufferUtil.bytes("col1")).setValue(ByteBufferUtil.bytes("val1")).setTimestamp(1), ConsistencyLevel.ONE);
-            Column column = tester.thriftClient.get(row, col, ConsistencyLevel.ONE).column;
-            System.out.println("read row " + new String(row.array()) + " " + new String(column.name.array()) + ":" + new String(column.value.array()) + ":" + column.timestamp);
-        }
-
-        System.exit(1);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
index 78b372e..b1249a6 100644
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -31,10 +31,6 @@ import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.*;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.ColumnDef;
-import org.apache.cassandra.thrift.IndexType;
-import org.apache.cassandra.thrift.ThriftConversion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -50,27 +46,6 @@ public class CFMetaDataTest
     private static final String KEYSPACE1 = "CFMetaDataTest1";
     private static final String CF_STANDARD1 = "Standard1";
 
-    private static List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();
-
-    static
-    {
-        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col1"), AsciiType.class.getCanonicalName())
-                                    .setIndex_name("col1Index")
-                                    .setIndex_type(IndexType.KEYS));
-
-        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col2"), UTF8Type.class.getCanonicalName())
-                                    .setIndex_name("col2Index")
-                                    .setIndex_type(IndexType.KEYS));
-
-        Map<String, String> customIndexOptions = new HashMap<>();
-        customIndexOptions.put("option1", "value1");
-        customIndexOptions.put("option2", "value2");
-        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col3"), Int32Type.class.getCanonicalName())
-                                    .setIndex_name("col3Index")
-                                    .setIndex_type(IndexType.CUSTOM)
-                                    .setIndex_options(customIndexOptions));
-    }
-
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
@@ -81,45 +56,6 @@ public class CFMetaDataTest
     }
 
     @Test
-    public void testThriftConversion() throws Exception
-    {
-        CfDef cfDef = new CfDef().setDefault_validation_class(AsciiType.class.getCanonicalName())
-                                 .setComment("Test comment")
-                                 .setColumn_metadata(columnDefs)
-                                 .setKeyspace(KEYSPACE1)
-                                 .setName(CF_STANDARD1);
-
-        // convert Thrift to CFMetaData
-        CFMetaData cfMetaData = ThriftConversion.fromThrift(cfDef);
-
-        CfDef thriftCfDef = new CfDef();
-        thriftCfDef.keyspace = KEYSPACE1;
-        thriftCfDef.name = CF_STANDARD1;
-        thriftCfDef.default_validation_class = cfDef.default_validation_class;
-        thriftCfDef.comment = cfDef.comment;
-        thriftCfDef.column_metadata = new ArrayList<>();
-        for (ColumnDef columnDef : columnDefs)
-        {
-            ColumnDef c = new ColumnDef();
-            c.name = ByteBufferUtil.clone(columnDef.name);
-            c.validation_class = columnDef.getValidation_class();
-            c.index_name = columnDef.getIndex_name();
-            c.index_type = columnDef.getIndex_type();
-            if (columnDef.isSetIndex_options())
-                c.setIndex_options(columnDef.getIndex_options());
-            thriftCfDef.column_metadata.add(c);
-        }
-
-        CfDef converted = ThriftConversion.toThrift(cfMetaData);
-
-        assertEquals(thriftCfDef.keyspace, converted.keyspace);
-        assertEquals(thriftCfDef.name, converted.name);
-        assertEquals(thriftCfDef.default_validation_class, converted.default_validation_class);
-        assertEquals(thriftCfDef.comment, converted.comment);
-        assertEquals(new HashSet<>(thriftCfDef.column_metadata), new HashSet<>(converted.column_metadata));
-    }
-
-    @Test
     public void testConversionsInverses() throws Exception
     {
         for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
@@ -127,9 +63,6 @@ public class CFMetaDataTest
             for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
             {
                 CFMetaData cfm = cfs.metadata;
-                if (!cfm.isThriftCompatible())
-                    continue;
-
                 checkInverses(cfm);
 
                 // Testing with compression to catch #3558
@@ -144,11 +77,6 @@ public class CFMetaDataTest
     {
         KeyspaceMetadata keyspace = Schema.instance.getKSMetaData(cfm.ksName);
 
-        // Test thrift conversion
-        CFMetaData before = cfm;
-        CFMetaData after = ThriftConversion.fromThriftForUpdate(ThriftConversion.toThrift(before), before);
-        assert before.equals(after) : String.format("%n%s%n!=%n%s", before, after);
-
         // Test schema conversion
         Mutation rm = SchemaKeyspace.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros()).build();
         PartitionUpdate cfU = rm.getPartitionUpdate(Schema.instance.getId(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.TABLES));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java b/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
deleted file mode 100644
index 1e8e704..0000000
--- a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package org.apache.cassandra.config;
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- *
- */
-
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.thrift.ThriftConversion;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class ColumnDefinitionTest
-{
-    @BeforeClass
-    public static void setupDD()
-    {
-        DatabaseDescriptor.daemonInitialization();
-    }
-
-    @Test
-    public void testSerializeDeserialize() throws Exception
-    {
-        CFMetaData cfm = CFMetaData.Builder.create("ks", "cf", true, false, false)
-                         .addPartitionKey("pkey", AsciiType.instance)
-                         .addClusteringColumn("name", AsciiType.instance)
-                         .addRegularColumn("val", AsciiType.instance)
-                         .build();
-
-        ColumnDefinition cd0 = ColumnDefinition.staticDef(cfm, ByteBufferUtil.bytes("TestColumnDefinitionName0"), BytesType.instance);
-        ColumnDefinition cd1 = ColumnDefinition.staticDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition1"), LongType.instance);
-
-        testSerializeDeserialize(cfm, cd0);
-        testSerializeDeserialize(cfm, cd1);
-    }
-
-    protected void testSerializeDeserialize(CFMetaData cfm, ColumnDefinition cd) throws Exception
-    {
-        ColumnDefinition newCd = ThriftConversion.fromThrift(cfm.ksName, cfm.cfName, cfm.comparator.subtype(0), null, ThriftConversion.toThrift(cfm, cd));
-        Assert.assertNotSame(cd, newCd);
-        Assert.assertEquals(cd.hashCode(), newCd.hashCode());
-        Assert.assertEquals(cd, newCd);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
index 421142f..17cdd77 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
@@ -62,7 +62,6 @@ public class DatabaseDescriptorRefTest
     "org.apache.cassandra.config.ConfigurationLoader",
     "org.apache.cassandra.config.Config",
     "org.apache.cassandra.config.Config$1",
-    "org.apache.cassandra.config.Config$RequestSchedulerId",
     "org.apache.cassandra.config.Config$CommitLogSync",
     "org.apache.cassandra.config.Config$DiskAccessMode",
     "org.apache.cassandra.config.Config$DiskFailurePolicy",
@@ -71,7 +70,6 @@ public class DatabaseDescriptorRefTest
     "org.apache.cassandra.config.Config$InternodeCompression",
     "org.apache.cassandra.config.Config$MemtableAllocationType",
     "org.apache.cassandra.config.Config$UserFunctionTimeoutPolicy",
-    "org.apache.cassandra.config.RequestSchedulerOptions",
     "org.apache.cassandra.config.ParameterizedClass",
     "org.apache.cassandra.config.EncryptionOptions",
     "org.apache.cassandra.config.EncryptionOptions$ClientEncryptionOptions",
@@ -103,7 +101,6 @@ public class DatabaseDescriptorRefTest
     "org.apache.cassandra.locator.SimpleSeedProvider",
     "org.apache.cassandra.locator.SeedProvider",
     "org.apache.cassandra.net.BackPressureStrategy",
-    "org.apache.cassandra.scheduler.IRequestScheduler",
     "org.apache.cassandra.security.EncryptionContext",
     "org.apache.cassandra.service.CacheService$CacheType",
     "org.apache.cassandra.utils.FBUtilities",
@@ -201,7 +198,6 @@ public class DatabaseDescriptorRefTest
         for (String methodName : new String[]{
             "clientInitialization",
             "applyAddressConfig",
-            "applyThriftHSHA",
             "applyInitialTokens",
             // no seed provider in default configuration for clients
             // "applySeedProvider",
@@ -211,7 +207,6 @@ public class DatabaseDescriptorRefTest
             // "applySnitch",
             "applyEncryptionContext",
             // starts "REQUEST-SCHEDULER" thread via RoundRobinScheduler
-            // "applyRequestScheduler",
         })
         {
             Method method = cDatabaseDescriptor.getDeclaredMethod(methodName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 359ef53..bce9182 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -39,7 +39,6 @@ import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.thrift.ThriftConversion;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -56,34 +55,6 @@ public class DatabaseDescriptorTest
         DatabaseDescriptor.daemonInitialization();
     }
 
-    @Test
-    public void testCFMetaDataSerialization() throws ConfigurationException, InvalidRequestException
-    {
-        // test serialization of all defined test CFs.
-        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
-        {
-            for (CFMetaData cfm : Schema.instance.getTablesAndViews(keyspaceName))
-            {
-                CFMetaData cfmDupe = ThriftConversion.fromThrift(ThriftConversion.toThrift(cfm));
-                assertNotNull(cfmDupe);
-                assertEquals(cfm, cfmDupe);
-            }
-        }
-    }
-
-    @Test
-    public void testKSMetaDataSerialization() throws ConfigurationException
-    {
-        for (String ks : Schema.instance.getNonSystemKeyspaces())
-        {
-            // Not testing round-trip on the KsDef via serDe() because maps
-            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ks);
-            KeyspaceMetadata ksmDupe = ThriftConversion.fromThrift(ThriftConversion.toThrift(ksm));
-            assertNotNull(ksmDupe);
-            assertEquals(ksm, ksmDupe);
-        }
-    }
-
     // this came as a result of CASSANDRA-995
     @Test
     public void testTransKsMigration() throws ConfigurationException, IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 04ee5df..a3e2021 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -760,7 +760,7 @@ public abstract class CQLTester
 
     protected ResultMessage.Prepared prepare(String query) throws Throwable
     {
-        return QueryProcessor.prepare(formatQuery(query), ClientState.forInternalCalls(), false);
+        return QueryProcessor.prepare(formatQuery(query), ClientState.forInternalCalls());
     }
 
     protected UntypedResultSet execute(String query, Object... values) throws Throwable

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
index 380dbda..0bcb877 100644
--- a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
@@ -56,16 +56,16 @@ public class PstmtPersistenceTest extends CQLTester
 
         List<MD5Digest> stmtIds = new ArrayList<>();
         // #0
-        stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + SchemaConstants.SCHEMA_KEYSPACE_NAME + '.' + SchemaKeyspace.TABLES + " WHERE keyspace_name = ?", clientState, false).statementId);
+        stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + SchemaConstants.SCHEMA_KEYSPACE_NAME + '.' + SchemaKeyspace.TABLES + " WHERE keyspace_name = ?", clientState).statementId);
         // #1
-        stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + KEYSPACE + '.' + currentTable() + " WHERE pk = ?", clientState, false).statementId);
+        stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + KEYSPACE + '.' + currentTable() + " WHERE pk = ?", clientState).statementId);
         // #2
-        stmtIds.add(QueryProcessor.prepare("SELECT * FROM foo.bar WHERE key = ?", clientState, false).statementId);
+        stmtIds.add(QueryProcessor.prepare("SELECT * FROM foo.bar WHERE key = ?", clientState).statementId);
         clientState.setKeyspace("foo");
         // #3
-        stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + KEYSPACE + '.' + currentTable() + " WHERE pk = ?", clientState, false).statementId);
+        stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + KEYSPACE + '.' + currentTable() + " WHERE pk = ?", clientState).statementId);
         // #4
-        stmtIds.add(QueryProcessor.prepare("SELECT * FROM foo.bar WHERE key = ?", clientState, false).statementId);
+        stmtIds.add(QueryProcessor.prepare("SELECT * FROM foo.bar WHERE key = ?", clientState).statementId);
 
         Assert.assertEquals(5, stmtIds.size());
         Assert.assertEquals(5, QueryProcessor.preparedStatementsCount());
@@ -97,7 +97,7 @@ public class PstmtPersistenceTest extends CQLTester
         }
 
         // add anther prepared statement and sync it to table
-        QueryProcessor.prepare("SELECT * FROM bar WHERE key = ?", clientState, false);
+        QueryProcessor.prepare("SELECT * FROM bar WHERE key = ?", clientState);
         Assert.assertEquals(6, QueryProcessor.preparedStatementsCount());
         rows = QueryProcessor.executeOnceInternal(queryAll).size();
         Assert.assertEquals(6, rows);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
deleted file mode 100644
index 521c0a0..0000000
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ /dev/null
@@ -1,110 +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.cql3;
-
-import java.util.Arrays;
-import java.util.Collections;
-
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.Int32Type;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.ColumnDef;
-import org.apache.cassandra.thrift.ThriftConversion;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import static junit.framework.Assert.assertFalse;
-import static junit.framework.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-
-public class ThriftCompatibilityTest extends SchemaLoader
-{
-    @Test // test for CASSANDRA-8178
-    public void testNonTextComparator() throws Throwable
-    {
-        ColumnDef column = new ColumnDef();
-        column.setName(bytes(42))
-              .setValidation_class(UTF8Type.instance.toString());
-
-        CfDef cf = new CfDef("thriftcompat", "JdbcInteger");
-        cf.setColumn_type("Standard")
-          .setComparator_type(Int32Type.instance.toString())
-          .setDefault_validation_class(UTF8Type.instance.toString())
-          .setKey_validation_class(BytesType.instance.toString())
-          .setColumn_metadata(Collections.singletonList(column));
-
-        SchemaLoader.createKeyspace("thriftcompat", KeyspaceParams.simple(1), ThriftConversion.fromThrift(cf));
-
-        // the comparator is IntegerType, and there is a column named 42 with a UTF8Type validation type
-        execute("INSERT INTO \"thriftcompat\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001, 'abc')");
-        execute("UPDATE \"thriftcompat\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key = 0x00000001");
-        execute("DELETE \"42\" FROM \"thriftcompat\".\"JdbcInteger\" WHERE key = 0x00000000");
-        UntypedResultSet results = execute("SELECT key, \"42\" FROM \"thriftcompat\".\"JdbcInteger\"");
-        assertEquals(1, results.size());
-        UntypedResultSet.Row row = results.iterator().next();
-        assertEquals(ByteBufferUtil.bytes(1), row.getBytes("key"));
-        assertEquals("abc", row.getString("42"));
-    }
-
-    @Test // test for CASSANDRA-9867
-    public void testDropCompactStaticColumn()
-    {
-        ColumnDef column1 = new ColumnDef();
-        column1.setName(bytes(42))
-              .setValidation_class(UTF8Type.instance.toString());
-
-        ColumnDef column2 = new ColumnDef();
-        column2.setName(bytes(25))
-               .setValidation_class(UTF8Type.instance.toString());
-
-        CfDef cf = new CfDef("thriftks", "staticcompact");
-        cf.setColumn_type("Standard")
-          .setComparator_type(Int32Type.instance.toString())
-          .setDefault_validation_class(UTF8Type.instance.toString())
-          .setKey_validation_class(BytesType.instance.toString())
-          .setColumn_metadata(Arrays.asList(column1, column2));
-
-        SchemaLoader.createKeyspace("thriftks", KeyspaceParams.simple(1), ThriftConversion.fromThrift(cf));
-        CFMetaData cfm = Schema.instance.getCFMetaData("thriftks", "staticcompact");
-
-        // assert the both columns are in the metadata
-        assertTrue(cfm.getColumnMetadata().containsKey(bytes(42)));
-        assertTrue(cfm.getColumnMetadata().containsKey(bytes(25)));
-
-        // remove column2
-        cf.setColumn_metadata(Collections.singletonList(column1));
-        MigrationManager.announceColumnFamilyUpdate(ThriftConversion.fromThriftForUpdate(cf, cfm), true);
-
-        // assert that it's gone from metadata
-        assertTrue(cfm.getColumnMetadata().containsKey(bytes(42)));
-        assertFalse(cfm.getColumnMetadata().containsKey(bytes(25)));
-    }
-
-    private static UntypedResultSet execute(String query)
-    {
-        return QueryProcessor.executeInternal(query);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 5af3ebb..fcd56f8 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -1029,16 +1029,13 @@ public class SecondaryIndexTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY ((a), b))");
         createIndex("CREATE INDEX c_idx ON %s(c)");
-        MD5Digest cqlId = prepareStatement("SELECT * FROM %s.%s WHERE c=?", false).statementId;
-        Integer thriftId = prepareStatement("SELECT * FROM %s.%s WHERE c=?", true).toThriftPreparedResult().getItemId();
+        MD5Digest cqlId = prepareStatement("SELECT * FROM %s.%s WHERE c=?").statementId;
 
         assertNotNull(QueryProcessor.instance.getPrepared(cqlId));
-        assertNotNull(QueryProcessor.instance.getPreparedForThrift(thriftId));
 
         dropIndex("DROP INDEX %s.c_idx");
 
         assertNull(QueryProcessor.instance.getPrepared(cqlId));
-        assertNull(QueryProcessor.instance.getPreparedForThrift(thriftId));
     }
 
     // See CASSANDRA-11021
@@ -1289,11 +1286,10 @@ public class SecondaryIndexTest extends CQLTester
         });
     }
 
-    private ResultMessage.Prepared prepareStatement(String cql, boolean forThrift)
+    private ResultMessage.Prepared prepareStatement(String cql)
     {
         return QueryProcessor.prepare(String.format(cql, KEYSPACE, currentTable()),
-                                      ClientState.forInternalCalls(),
-                                      forThrift);
+                                      ClientState.forInternalCalls());
     }
 
     private void validateCell(Cell cell, ColumnDefinition def, ByteBuffer val, long timestamp)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index c54982e..74f6409 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -159,16 +159,16 @@ public class UFTest extends CQLTester
 
         ResultMessage.Prepared preparedSelect1 = QueryProcessor.prepare(
                                                                        String.format("SELECT key, %s(d) FROM %s.%s", fSin, KEYSPACE, currentTable()),
-                                                                       ClientState.forInternalCalls(), false);
+                                                                       ClientState.forInternalCalls());
         ResultMessage.Prepared preparedSelect2 = QueryProcessor.prepare(
                                                     String.format("SELECT key FROM %s.%s", KEYSPACE, currentTable()),
-                                                    ClientState.forInternalCalls(), false);
+                                                    ClientState.forInternalCalls());
         ResultMessage.Prepared preparedInsert1 = QueryProcessor.prepare(
                                                       String.format("INSERT INTO %s.%s (key, d) VALUES (?, %s(?))", KEYSPACE, currentTable(), fSin),
-                                                      ClientState.forInternalCalls(), false);
+                                                      ClientState.forInternalCalls());
         ResultMessage.Prepared preparedInsert2 = QueryProcessor.prepare(
                                                       String.format("INSERT INTO %s.%s (key, d) VALUES (?, ?)", KEYSPACE, currentTable()),
-                                                      ClientState.forInternalCalls(), false);
+                                                      ClientState.forInternalCalls());
 
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
@@ -193,10 +193,10 @@ public class UFTest extends CQLTester
 
         preparedSelect1= QueryProcessor.prepare(
                                          String.format("SELECT key, %s(d) FROM %s.%s", fSin, KEYSPACE, currentTable()),
-                                         ClientState.forInternalCalls(), false);
+                                         ClientState.forInternalCalls());
         preparedInsert1 = QueryProcessor.prepare(
                                          String.format("INSERT INTO %s.%s (key, d) VALUES (?, %s(?))", KEYSPACE, currentTable(), fSin),
-                                         ClientState.forInternalCalls(), false);
+                                         ClientState.forInternalCalls());
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
 
@@ -262,7 +262,7 @@ public class UFTest extends CQLTester
                                                                              KEYSPACE,
                                                                              currentTable(),
                                                                              literalArgs),
-                                                                ClientState.forInternalCalls(), false);
+                                                                ClientState.forInternalCalls());
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
         return prepared;
     }
@@ -278,7 +278,7 @@ public class UFTest extends CQLTester
                                                                              KEYSPACE,
                                                                              currentTable(),
                                                                              function),
-                                                                ClientState.forInternalCalls(), false);
+                                                                ClientState.forInternalCalls());
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
         return prepared;
     }
@@ -293,7 +293,7 @@ public class UFTest extends CQLTester
                                                                String.format("INSERT INTO %s.%s (key, val) VALUES (?, ?)",
                                                                             KEYSPACE,
                                                                             currentTable()),
-                                                               ClientState.forInternalCalls(), false);
+                                                               ClientState.forInternalCalls());
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(control.statementId));
 
         // a function that we'll drop and verify that statements which use it to
@@ -741,7 +741,7 @@ public class UFTest extends CQLTester
         Assert.assertEquals(1, Schema.instance.getFunctions(fNameName).size());
 
         ResultMessage.Prepared prepared = QueryProcessor.prepare(String.format("SELECT key, %s(udt) FROM %s.%s", fName, KEYSPACE, currentTable()),
-                                                                 ClientState.forInternalCalls(), false);
+                                                                 ClientState.forInternalCalls());
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
 
         // UT still referenced by table

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
index 90887fb..cfedc08 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -1073,7 +1073,7 @@ public class AggregationTest extends CQLTester
                                        "SFUNC " + shortFunctionName(fState) + " " +
                                        "STYPE int");
 
-            ResultMessage.Prepared prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls(), false);
+            ResultMessage.Prepared prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls());
             assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
 
             execute("DROP AGGREGATE " + a + "(int)");
@@ -1085,7 +1085,7 @@ public class AggregationTest extends CQLTester
                     "SFUNC " + shortFunctionName(fState) + " " +
                     "STYPE int");
 
-            prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls(), false);
+            prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls());
             assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
 
             execute("DROP KEYSPACE " + otherKS + ";");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
index 605876f..5c1f865 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -51,13 +51,6 @@ import static org.junit.Assert.assertEquals;
 public class CreateTest extends CQLTester
 {
     @Test
-    public void testCQL3PartitionKeyOnlyTable()
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
-        assertFalse(currentTableMetadata().isThriftCompatible());
-    }
-
-    @Test
     public void testCreateTableWithSmallintColumns() throws Throwable
     {
         createTable("CREATE TABLE %s (a text, b smallint, c smallint, primary key (a, b));");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 8463a1f..a2003d8 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -157,93 +157,6 @@ public class ColumnFamilyStoreTest
         reTest(cfs, r);
     }
 
-    // TODO: Implement this once we have hooks to super columns available in CQL context
-//    @Test
-//    public void testDeleteSuperRowSticksAfterFlush() throws Throwable
-//    {
-//        String keyspaceName = KEYSPACE1;
-//        String cfName= CF_SUPER1;
-//
-//        Keyspace keyspace = Keyspace.open(keyspaceName);
-//        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
-//
-//        ByteBuffer scfName = ByteBufferUtil.bytes("SuperDuper");
-//        DecoratedKey key = Util.dk("flush-resurrection");
-//
-//        // create an isolated sstable.
-//        putColSuper(cfs, key, 0, ByteBufferUtil.bytes("val"), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes("val1"));
-
-//        putColsSuper(cfs, key, scfName,
-//                new BufferCell(cellname(1L), ByteBufferUtil.bytes("val1"), 1),
-//                new BufferCell(cellname(2L), ByteBufferUtil.bytes("val2"), 1),
-//                new BufferCell(cellname(3L), ByteBufferUtil.bytes("val3"), 1));
-//        cfs.forceBlockingFlush();
-//
-//        // insert, don't flush.
-//        putColsSuper(cfs, key, scfName,
-//                new BufferCell(cellname(4L), ByteBufferUtil.bytes("val4"), 1),
-//                new BufferCell(cellname(5L), ByteBufferUtil.bytes("val5"), 1),
-//                new BufferCell(cellname(6L), ByteBufferUtil.bytes("val6"), 1));
-//
-//        // verify insert.
-//        final SlicePredicate sp = new SlicePredicate();
-//        sp.setSlice_range(new SliceRange());
-//        sp.getSlice_range().setCount(100);
-//        sp.getSlice_range().setStart(ArrayUtils.EMPTY_BYTE_ARRAY);
-//        sp.getSlice_range().setFinish(ArrayUtils.EMPTY_BYTE_ARRAY);
-//
-//        assertRowAndColCount(1, 6, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
-//
-//        // delete
-//        Mutation rm = new Mutation(keyspace.getName(), key.getKey());
-//        rm.deleteRange(cfName, SuperColumns.startOf(scfName), SuperColumns.endOf(scfName), 2);
-//        rm.applyUnsafe();
-//
-//        // verify delete.
-//        assertRowAndColCount(1, 0, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
-//
-//        // flush
-//        cfs.forceBlockingFlush();
-//
-//        // re-verify delete.
-//        assertRowAndColCount(1, 0, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
-//
-//        // late insert.
-//        putColsSuper(cfs, key, scfName,
-//                new BufferCell(cellname(4L), ByteBufferUtil.bytes("val4"), 1L),
-//                new BufferCell(cellname(7L), ByteBufferUtil.bytes("val7"), 1L));
-//
-//        // re-verify delete.
-//        assertRowAndColCount(1, 0, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
-//
-//        // make sure new writes are recognized.
-//        putColsSuper(cfs, key, scfName,
-//                new BufferCell(cellname(3L), ByteBufferUtil.bytes("val3"), 3),
-//                new BufferCell(cellname(8L), ByteBufferUtil.bytes("val8"), 3),
-//                new BufferCell(cellname(9L), ByteBufferUtil.bytes("val9"), 3));
-//        assertRowAndColCount(1, 3, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
-//    }
-
-//    private static void assertRowAndColCount(int rowCount, int colCount, boolean isDeleted, Collection<Row> rows) throws CharacterCodingException
-//    {
-//        assert rows.size() == rowCount : "rowcount " + rows.size();
-//        for (Row row : rows)
-//        {
-//            assert row.cf != null : "cf was null";
-//            assert row.cf.getColumnCount() == colCount : "colcount " + row.cf.getColumnCount() + "|" + str(row.cf);
-//            if (isDeleted)
-//                assert row.cf.isMarkedForDelete() : "cf not marked for delete";
-//        }
-//    }
-//
-//    private static String str(ColumnFamily cf) throws CharacterCodingException
-//    {
-//        StringBuilder sb = new StringBuilder();
-//        for (Cell col : cf.getSortedColumns())
-//            sb.append(String.format("(%s,%s,%d),", ByteBufferUtil.string(col.name().toByteBuffer()), ByteBufferUtil.string(col.value()), col.timestamp()));
-//        return sb.toString();
-//    }
-
     @Test
     public void testDeleteStandardRowSticksAfterFlush() throws Throwable
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java b/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java
deleted file mode 100644
index 8398ede..0000000
--- a/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java
+++ /dev/null
@@ -1,89 +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.db;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-
-import static junit.framework.Assert.assertTrue;
-
-public class LegacyCellNameTest
-{
-    @BeforeClass
-    public static void setupDD()
-    {
-        DatabaseDescriptor.daemonInitialization();
-    }
-
-    @Test
-    public void testColumnSameNameAsPartitionKeyCompactStorage() throws Exception
-    {
-        CFMetaData cfm = CFMetaData.compile("CREATE TABLE cs (" +
-                                            "k int PRIMARY KEY, v int)" +
-                                            " WITH COMPACT STORAGE", "ks");
-
-        LegacyLayout.LegacyCellName cellName 
-            = LegacyLayout.decodeCellName(cfm, 
-                                          LegacyLayout.makeLegacyComparator(cfm)
-                                                      .fromString("k"));
-
-        assertTrue(cellName.column.isRegular());
-    }
-
-    @Test
-    public void testColumnSameNameAsClusteringKeyCompactStorage() throws Exception
-    {
-        CFMetaData cfm = CFMetaData.compile("CREATE TABLE cs (" +
-                                            "k int PRIMARY KEY, v int)" +
-                                            " WITH COMPACT STORAGE", "ks");
-
-        LegacyLayout.LegacyCellName cellName 
-            = LegacyLayout.decodeCellName(cfm, 
-                                          LegacyLayout.makeLegacyComparator(cfm)
-                                                      .fromString("column1"));
-
-        assertTrue(cellName.column.isRegular());
-    }
-
-    @Test(expected=IllegalArgumentException.class)
-    public void testColumnSameNameAsPartitionKeyCql3() throws Exception
-    {
-        CFMetaData cfm = CFMetaData.compile("CREATE TABLE cs (" +
-                                            "k int PRIMARY KEY, v int)", "ks");
-
-        LegacyLayout.LegacyCellName cellName 
-            = LegacyLayout.decodeCellName(cfm, 
-                                          LegacyLayout.makeLegacyComparator(cfm)
-                                                      .fromString("k"));
-    }
-
-    @Test(expected=IllegalArgumentException.class)
-    public void testColumnSameNameAsClusteringKeyCql3() throws Exception
-    {
-        CFMetaData cfm = CFMetaData.compile("CREATE TABLE cs (" +
-                                            "k int, c text, v int, PRIMARY KEY(k, c))", "ks");
-
-        LegacyLayout.LegacyCellName cellName 
-            = LegacyLayout.decodeCellName(cfm, 
-                                          LegacyLayout.makeLegacyComparator(cfm)
-                                                      .fromString("c"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java b/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java
index d5fb8fa..e409592 100644
--- a/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java
+++ b/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java
@@ -155,211 +155,5 @@ public class PartitionRangeReadTest
         assertTrue(partitions.get(0).iterator().next().getCell(cDef).value().equals(ByteBufferUtil.bytes("2")));
         assertTrue(partitions.get(partitions.size() - 1).iterator().next().getCell(cDef).value().equals(ByteBufferUtil.bytes("6")));
     }
-
-        // TODO: Port or remove, depending on what DataLimits.thriftLimits (per cell) looks like
-//    @Test
-//    public void testRangeSliceColumnsLimit() throws Throwable
-//    {
-//        String keyspaceName = KEYSPACE1;
-//        String cfName = CF_STANDARD1;
-//        Keyspace keyspace = Keyspace.open(keyspaceName);
-//        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
-//        cfs.clearUnsafe();
-//
-//        Cell[] cols = new Cell[5];
-//        for (int i = 0; i < 5; i++)
-//            cols[i] = column("c" + i, "value", 1);
-//
-//        putColsStandard(cfs, Util.dk("a"), cols[0], cols[1], cols[2], cols[3], cols[4]);
-//        putColsStandard(cfs, Util.dk("b"), cols[0], cols[1]);
-//        putColsStandard(cfs, Util.dk("c"), cols[0], cols[1], cols[2], cols[3]);
-//        cfs.forceBlockingFlush();
-//
-//        SlicePredicate sp = new SlicePredicate();
-//        sp.setSlice_range(new SliceRange());
-//        sp.getSlice_range().setCount(1);
-//        sp.getSlice_range().setStart(ArrayUtils.EMPTY_BYTE_ARRAY);
-//        sp.getSlice_range().setFinish(ArrayUtils.EMPTY_BYTE_ARRAY);
-//
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              3,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            3);
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              5,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            5);
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              8,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            8);
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              10,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            10);
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              100,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            11);
-//
-//        // Check that when querying by name, we always include all names for a
-//        // gien row even if it means returning more columns than requested (this is necesseray for CQL)
-//        sp = new SlicePredicate();
-//        sp.setColumn_names(Arrays.asList(
-//            ByteBufferUtil.bytes("c0"),
-//            ByteBufferUtil.bytes("c1"),
-//            ByteBufferUtil.bytes("c2")
-//        ));
-//
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              1,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            3);
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              4,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            5);
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              5,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            5);
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              6,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            8);
-//        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""),
-//                                              null,
-//                                              ThriftValidation.asIFilter(sp, cfs.metadata, null),
-//                                              100,
-//                                              System.currentTimeMillis(),
-//                                              true,
-//                                              false),
-//                            8);
-//    }
-
-    // TODO: Port or remove, depending on what DataLimits.thriftLimits (per cell) looks like
-//    @Test
-//    public void testRangeSlicePaging() throws Throwable
-//    {
-//        String keyspaceName = KEYSPACE1;
-//        String cfName = CF_STANDARD1;
-//        Keyspace keyspace = Keyspace.open(keyspaceName);
-//        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
-//        cfs.clearUnsafe();
-//
-//        Cell[] cols = new Cell[4];
-//        for (int i = 0; i < 4; i++)
-//            cols[i] = column("c" + i, "value", 1);
-//
-//        DecoratedKey ka = Util.dk("a");
-//        DecoratedKey kb = Util.dk("b");
-//        DecoratedKey kc = Util.dk("c");
-//
-//        PartitionPosition min = Util.rp("");
-//
-//        putColsStandard(cfs, ka, cols[0], cols[1], cols[2], cols[3]);
-//        putColsStandard(cfs, kb, cols[0], cols[1], cols[2]);
-//        putColsStandard(cfs, kc, cols[0], cols[1], cols[2], cols[3]);
-//        cfs.forceBlockingFlush();
-//
-//        SlicePredicate sp = new SlicePredicate();
-//        sp.setSlice_range(new SliceRange());
-//        sp.getSlice_range().setCount(1);
-//        sp.getSlice_range().setStart(ArrayUtils.EMPTY_BYTE_ARRAY);
-//        sp.getSlice_range().setFinish(ArrayUtils.EMPTY_BYTE_ARRAY);
-//
-//        Collection<Row> rows;
-//        Row row, row1, row2;
-//        IDiskAtomFilter filter = ThriftValidation.asIFilter(sp, cfs.metadata, null);
-//
-//        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(Util.range("", ""), filter, null, 3, true, true, System.currentTimeMillis()));
-//        assert rows.size() == 1 : "Expected 1 row, got " + toString(rows);
-//        row = rows.iterator().next();
-//        assertColumnNames(row, "c0", "c1", "c2");
-//
-//        sp.getSlice_range().setStart(ByteBufferUtil.getArray(ByteBufferUtil.bytes("c2")));
-//        filter = ThriftValidation.asIFilter(sp, cfs.metadata, null);
-//        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<PartitionPosition>(ka, min), filter, null, 3, true, true, System.currentTimeMillis()));
-//        assert rows.size() == 2 : "Expected 2 rows, got " + toString(rows);
-//        Iterator<Row> iter = rows.iterator();
-//        row1 = iter.next();
-//        row2 = iter.next();
-//        assertColumnNames(row1, "c2", "c3");
-//        assertColumnNames(row2, "c0");
-//
-//        sp.getSlice_range().setStart(ByteBufferUtil.getArray(ByteBufferUtil.bytes("c0")));
-//        filter = ThriftValidation.asIFilter(sp, cfs.metadata, null);
-//        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<PartitionPosition>(row2.key, min), filter, null, 3, true, true, System.currentTimeMillis()));
-//        assert rows.size() == 1 : "Expected 1 row, got " + toString(rows);
-//        row = rows.iterator().next();
-//        assertColumnNames(row, "c0", "c1", "c2");
-//
-//        sp.getSlice_range().setStart(ByteBufferUtil.getArray(ByteBufferUtil.bytes("c2")));
-//        filter = ThriftValidation.asIFilter(sp, cfs.metadata, null);
-//        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<PartitionPosition>(row.key, min), filter, null, 3, true, true, System.currentTimeMillis()));
-//        assert rows.size() == 2 : "Expected 2 rows, got " + toString(rows);
-//        iter = rows.iterator();
-//        row1 = iter.next();
-//        row2 = iter.next();
-//        assertColumnNames(row1, "c2");
-//        assertColumnNames(row2, "c0", "c1");
-//
-//        // Paging within bounds
-//        SliceQueryFilter sf = new SliceQueryFilter(cellname("c1"),
-//                                                   cellname("c2"),
-//                                                   false,
-//                                                   0);
-//        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<PartitionPosition>(ka, kc), sf, cellname("c2"), cellname("c1"), null, 2, true, System.currentTimeMillis()));
-//        assert rows.size() == 2 : "Expected 2 rows, got " + toString(rows);
-//        iter = rows.iterator();
-//        row1 = iter.next();
-//        row2 = iter.next();
-//        assertColumnNames(row1, "c2");
-//        assertColumnNames(row2, "c1");
-//
-//        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<PartitionPosition>(kb, kc), sf, cellname("c1"), cellname("c1"), null, 10, true, System.currentTimeMillis()));
-//        assert rows.size() == 2 : "Expected 2 rows, got " + toString(rows);
-//        iter = rows.iterator();
-//        row1 = iter.next();
-//        row2 = iter.next();
-//        assertColumnNames(row1, "c1", "c2");
-//        assertColumnNames(row2, "c1");
-//    }
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
index 69e1423..28590d8 100644
--- a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
@@ -141,7 +141,7 @@ public class RowIndexEntryTest extends CQLTester
         SerializationHeader header = new SerializationHeader(true, cfMeta, cfMeta.partitionColumns(), EncodingStats.NO_STATS);
 
         // create C-11206 + old serializer instances
-        RowIndexEntry.IndexSerializer rieSerializer = new RowIndexEntry.Serializer(cfMeta, version, header);
+        RowIndexEntry.IndexSerializer rieSerializer = new RowIndexEntry.Serializer(version, header);
         Pre_C_11206_RowIndexEntry.Serializer oldSerializer = new Pre_C_11206_RowIndexEntry.Serializer(cfMeta, version, header);
 
         @SuppressWarnings({ "resource", "IOResourceOpenedButNotSafelyClosed" })

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
index 95582d5..f89aa8d 100644
--- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
@@ -120,7 +120,7 @@ public class SinglePartitionSliceCommandTest
 
         ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
+        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, cfm,
                                                          FBUtilities.nowInSeconds(),
                                                          columnFilter,
                                                          RowFilter.NONE,
@@ -178,7 +178,7 @@ public class SinglePartitionSliceCommandTest
         ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
         Slice slice = Slice.make(ClusteringBound.BOTTOM, ClusteringBound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfm.comparator, slice), false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
+        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, cfm,
                                                          FBUtilities.nowInSeconds(),
                                                          columnFilter,
                                                          RowFilter.NONE,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java
index dc5fd06..3c8c1b0 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java
@@ -339,12 +339,6 @@ public class CompactionIteratorTest
         }
 
         @Override
-        public boolean isForThrift()
-        {
-            return false;
-        }
-
-        @Override
         public CFMetaData metadata()
         {
             return metadata;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4881d9c3/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index e1622ff..55bff5e 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -238,7 +238,7 @@ public class TTLExpiryTest
         cfs.enableAutoCompaction(true);
         assertEquals(1, cfs.getLiveSSTables().size());
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
-        ISSTableScanner scanner = sstable.getScanner(ColumnFilter.all(sstable.metadata), DataRange.allData(cfs.getPartitioner()), false);
+        ISSTableScanner scanner = sstable.getScanner(ColumnFilter.all(sstable.metadata), DataRange.allData(cfs.getPartitioner()));
         assertTrue(scanner.hasNext());
         while(scanner.hasNext())
         {