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

[1/7] Support multi-row selects within a partition using IN

Repository: cassandra
Updated Branches:
  refs/heads/trunk 864865da9 -> e5ab470d5


http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
new file mode 100644
index 0000000..b728cba
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
@@ -0,0 +1,1112 @@
+/*
+ * 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 org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.MD5Digest;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import static org.apache.cassandra.cql3.QueryProcessor.process;
+import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static com.google.common.collect.Lists.newArrayList;
+import static org.junit.Assert.fail;
+
+public class MultiColumnRelationTest
+{
+    private static final Logger logger = LoggerFactory.getLogger(MultiColumnRelationTest.class);
+    static ClientState clientState;
+    static String keyspace = "multi_column_relation_test";
+
+    @BeforeClass
+    public static void setUpClass() throws Throwable
+    {
+        SchemaLoader.loadSchema();
+        executeSchemaChange("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_partition (a int PRIMARY KEY, b int)");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.compound_partition (a int, b int, c int, PRIMARY KEY ((a, b)))");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_clustering (a int, b int, c int, PRIMARY KEY (a, b))");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.multiple_clustering (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.multiple_clustering_reversed (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
+        clientState = ClientState.forInternalCalls();
+    }
+
+    @AfterClass
+    public static void stopGossiper()
+    {
+        Gossiper.instance.stop();
+    }
+
+    private static void executeSchemaChange(String query) throws Throwable
+    {
+        try
+        {
+            process(String.format(query, keyspace), ConsistencyLevel.ONE);
+        } catch (RuntimeException exc)
+        {
+            throw exc.getCause();
+        }
+    }
+
+    private static UntypedResultSet execute(String query) throws Throwable
+    {
+        try
+        {
+            return processInternal(String.format(query, keyspace));
+        } catch (RuntimeException exc)
+        {
+            if (exc.getCause() != null)
+                throw exc.getCause();
+            throw exc;
+        }
+    }
+
+    private MD5Digest prepare(String query) throws RequestValidationException
+    {
+        ResultMessage.Prepared prepared = QueryProcessor.prepare(String.format(query, keyspace), clientState, false);
+        return prepared.statementId;
+    }
+
+    private UntypedResultSet executePrepared(MD5Digest statementId, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    {
+        CQLStatement statement = QueryProcessor.instance.getPrepared(statementId);
+        ResultMessage message = statement.executeInternal(QueryState.forInternalCalls(), options);
+
+        if (message instanceof ResultMessage.Rows)
+            return new UntypedResultSet(((ResultMessage.Rows)message).result);
+        else
+            return null;
+    }
+
+    @Test(expected=SyntaxException.class)
+    public void testEmptyIdentifierTuple() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_clustering WHERE () = (1, 2)");
+    }
+
+    @Test(expected=SyntaxException.class)
+    public void testEmptyValueTuple() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > ()");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testDifferentTupleLengths() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (1, 2, 3)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testNullInTuple() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (1, null)");
+    }
+
+    @Test
+    public void testEmptyIN() throws Throwable
+    {
+        UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ()");
+        assertTrue(results.isEmpty());
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testNullInINValues() throws Throwable
+    {
+        UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((1, 2, null))");
+        assertTrue(results.isEmpty());
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPartitionKeyInequality() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_partition WHERE (a) > (1)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPartitionKeyEquality() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_partition WHERE (a) = (0)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testRestrictNonPrimaryKey() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_partition WHERE (b) = (0)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testMixEqualityAndInequality() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (0) AND (b) > (0)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testMixMultipleInequalitiesOnSameBound() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0) AND (b) > (1)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testClusteringColumnsOutOfOrderInInequality() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) > (0, 0, 0)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testSkipClusteringColumnInEquality() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) = (0, 0)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testSkipClusteringColumnInInequality() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) > (0, 0)");
+    }
+
+    @Test
+    public void testSingleClusteringColumnEquality() throws Throwable
+    {
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)");
+        UntypedResultSet results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (1)");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0);
+
+        results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (3)");
+        assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testMultipleClusteringColumnEquality() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 2, 0, 0)");
+        UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) = (1)");
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(2, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) = (1, 1)");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 1, 0);
+        checkRow(1, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) = (1, 1, 1)");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 1, 1);
+        execute("DELETE FROM %s.multiple_clustering WHERE a=0 AND b=2 and c=0 and d=0");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPartitionAndClusteringColumnEquality() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_clustering WHERE (a, b) = (0, 0)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testClusteringColumnsOutOfOrderInEquality() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) = (3, 2, 1)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testBadType() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) = (1, 2, 'foobar')");
+    }
+
+    @Test(expected=SyntaxException.class)
+    public void testSingleColumnTupleRelation() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND b = (1, 2, 3)");
+    }
+
+    @Test
+    public void testMixSingleAndTupleInequalities() throws Throwable
+    {
+        String[] queries = new String[]{
+            "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND b < 1",
+            "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND c < 1",
+            "SELECT * FROM %s.multiple_clustering WHERE a=0 AND b > 1 AND (b, c, d) < (1, 1, 0)",
+            "SELECT * FROM %s.multiple_clustering WHERE a=0 AND c > 1 AND (b, c, d) < (1, 1, 0)",
+        };
+
+        for (String query : queries)
+        {
+            try
+            {
+                execute(query);
+                fail(String.format("Expected query \"%s\" to throw an InvalidRequestException", query));
+            }
+            catch (InvalidRequestException e) {}
+        }
+    }
+
+    @Test
+    public void testSingleClusteringColumnInequality() throws Throwable
+    {
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)");
+
+        UntypedResultSet results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0)");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 0);
+        checkRow(1, results, 0, 2, 0);
+
+        results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= (1)");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 0);
+        checkRow(1, results, 0, 2, 0);
+
+        results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < (2)");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 0);
+        checkRow(1, results, 0, 1, 0);
+
+        results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= (1)");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 0);
+        checkRow(1, results, 0, 1, 0);
+
+        results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0) AND (b) < (2)");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0);
+    }
+
+    @Test
+    public void testMultipleClusteringColumnInequality() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)");
+
+        UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (0)");
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(2, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) >= (0)");
+        assertEquals(6, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+        checkRow(3, results, 0, 1, 0, 0);
+        checkRow(4, results, 0, 1, 1, 0);
+        checkRow(5, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (1, 0)");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 1, 0);
+        checkRow(1, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) >= (1, 0)");
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(2, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (1, 1, 0)");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) >= (1, 1, 0)");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 1, 0);
+        checkRow(1, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) < (1)");
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) <= (1)");
+        assertEquals(6, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+        checkRow(3, results, 0, 1, 0, 0);
+        checkRow(4, results, 0, 1, 1, 0);
+        checkRow(5, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) < (0, 1)");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) <= (0, 1)");
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) < (0, 1, 1)");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) <= (0, 1, 1)");
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND (b) < (1)");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c) < (1, 1)");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c, d) < (1, 1, 0)");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+
+        // reversed
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (0) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(3, results.size());
+        checkRow(2, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) >= (0) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(6, results.size());
+        checkRow(5, results, 0, 0, 0, 0);
+        checkRow(4, results, 0, 0, 1, 0);
+        checkRow(3, results, 0, 0, 1, 1);
+        checkRow(2, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (1, 0) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(2, results.size());
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) >= (1, 0) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(3, results.size());
+        checkRow(2, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (1, 1, 0) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) >= (1, 1, 0) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(2, results.size());
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) < (1) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(3, results.size());
+        checkRow(2, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(0, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) <= (1) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(6, results.size());
+        checkRow(5, results, 0, 0, 0, 0);
+        checkRow(4, results, 0, 0, 1, 0);
+        checkRow(3, results, 0, 0, 1, 1);
+        checkRow(2, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) < (0, 1) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) <= (0, 1) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(3, results.size());
+        checkRow(2, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(0, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) < (0, 1, 1) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(2, results.size());
+        checkRow(1, results, 0, 0, 0, 0);
+        checkRow(0, results, 0, 0, 1, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) <= (0, 1, 1) ORDER BY b DESC, c DESC, d DESC");
+        checkRow(2, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(0, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND (b) < (1) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c) < (1, 1) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c, d) < (1, 1, 0) ORDER BY b DESC, c DESC, d DESC");
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+    }
+
+    @Test
+    public void testMultipleClusteringColumnInequalityReversedComponents() throws Throwable
+    {
+        // b and d are reversed in the clustering order
+        execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 1, 1)");
+        execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 1, 0)");
+
+        execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 1, 1)");
+        execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 1, 0)");
+
+
+        UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) > (0)");
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 1);
+        checkRow(2, results, 0, 1, 1, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) >= (0)");
+        assertEquals(6, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 1);
+        checkRow(2, results, 0, 1, 1, 0);
+        checkRow(3, results, 0, 0, 0, 0);
+        checkRow(4, results, 0, 0, 1, 1);
+        checkRow(5, results, 0, 0, 1, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) < (1)");
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+        checkRow(2, results, 0, 0, 1, 0);
+
+        results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) <= (1)");
+        assertEquals(6, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 1);
+        checkRow(2, results, 0, 1, 1, 0);
+        checkRow(3, results, 0, 0, 0, 0);
+        checkRow(4, results, 0, 0, 1, 1);
+        checkRow(5, results, 0, 0, 1, 0);
+
+        // preserve pre-6875 behavior (even though the query result is technically incorrect)
+        results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b, c) > (1, 0)");
+        assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testLiteralIn() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+        UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 0), (0, 1, 1))");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        // same query, but reversed order for the IN values
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 1), (0, 1, 0))");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN ((0, 1))");
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ((0))");
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testLiteralInWithShortTuple() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1))");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testLiteralInWithLongTuple() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 2, 3, 4))");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testLiteralInWithPartitionKey() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE (a, b, c, d) IN ((0, 1, 2, 3))");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testLiteralInSkipsClusteringColumn() throws Throwable
+    {
+        execute("SELECT * FROM %s.multiple_clustering WHERE (c, d) IN ((0, 1))");
+    }
+    @Test
+    public void testPartitionAndClusteringInClauses() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 1, 1)");
+
+        UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) AND (b, c, d) IN ((0, 1, 0), (0, 1, 1))");
+        assertEquals(4, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+        checkRow(2, results, 1, 0, 1, 0);
+        checkRow(3, results, 1, 0, 1, 1);
+
+        // same query, but reversed order for the IN values
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (1, 0) AND (b, c, d) IN ((0, 1, 1), (0, 1, 0))");
+        assertEquals(4, results.size());
+        checkRow(0, results, 1, 0, 1, 0);
+        checkRow(1, results, 1, 0, 1, 1);
+        checkRow(2, results, 0, 0, 1, 0);
+        checkRow(3, results, 0, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) and (b, c) IN ((0, 1))");
+        assertEquals(4, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+        checkRow(2, results, 1, 0, 1, 0);
+        checkRow(3, results, 1, 0, 1, 1);
+
+        results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) and (b) IN ((0))");
+        assertEquals(6, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+        checkRow(3, results, 1, 0, 0, 0);
+        checkRow(4, results, 1, 0, 1, 0);
+        checkRow(5, results, 1, 0, 1, 1);
+    }
+
+    // prepare statement tests
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPreparePartitionAndClusteringColumnEquality() throws Throwable
+    {
+        prepare("SELECT * FROM %s.single_clustering WHERE (a, b) = (?, ?)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareDifferentTupleLengths() throws Throwable
+    {
+        prepare("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (?, ?, ?)");
+    }
+
+    @Test
+    public void testPrepareEmptyIN() throws Throwable
+    {
+        MD5Digest id = prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ()");
+        UntypedResultSet results = executePrepared(id, makeIntOptions());
+        assertTrue(results.isEmpty());
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPreparePartitionKeyInequality() throws Throwable
+    {
+        prepare("SELECT * FROM %s.single_partition WHERE (a) > (?)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPreparePartitionKeyEquality() throws Throwable
+    {
+        prepare("SELECT * FROM %s.single_partition WHERE (a) = (?)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareRestrictNonPrimaryKey() throws Throwable
+    {
+        prepare("SELECT * FROM %s.single_partition WHERE (b) = (?)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareMixEqualityAndInequality() throws Throwable
+    {
+        prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (?) AND (b) > (?)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareMixMultipleInequalitiesOnSameBound() throws Throwable
+    {
+        prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?) AND (b) > (?)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareClusteringColumnsOutOfOrderInInequality() throws Throwable
+    {
+        prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) > (?, ?, ?)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareSkipClusteringColumnInEquality() throws Throwable
+    {
+        prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) = (?, ?)");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareSkipClusteringColumnInInequality() throws Throwable
+    {
+        prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) > (?, ?)");
+    }
+
+    @Test
+    public void testPreparedClusteringColumnEquality() throws Throwable
+    {
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+        MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (?)");
+        UntypedResultSet results = executePrepared(id, makeIntOptions(0));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 0, 0);
+    }
+
+    @Test
+    public void testPreparedClusteringColumnEqualitySingleMarker() throws Throwable
+    {
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+        MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = ?");
+        UntypedResultSet results = executePrepared(id, options(tuple(0)));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 0, 0);
+    }
+
+    @Test
+    public void testPreparedSingleClusteringColumnInequality() throws Throwable
+    {
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)");
+
+        MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?)");
+        UntypedResultSet results = executePrepared(id, makeIntOptions(0));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 0);
+        checkRow(1, results, 0, 2, 0);
+
+        results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= (?)"), makeIntOptions(1));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 0);
+        checkRow(1, results, 0, 2, 0);
+
+        results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < (?)"), makeIntOptions(2));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 0);
+        checkRow(1, results, 0, 1, 0);
+
+        results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= (?)"), makeIntOptions(1));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 0);
+        checkRow(1, results, 0, 1, 0);
+
+        results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?) AND (b) < (?)"), makeIntOptions(0, 2));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0);
+    }
+
+    @Test
+    public void testPreparedSingleClusteringColumnInequalitySingleMarker() throws Throwable
+    {
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+        execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)");
+
+        MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > ?");
+        UntypedResultSet results = executePrepared(id, options(tuple(0)));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 0);
+        checkRow(1, results, 0, 2, 0);
+
+        results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= ?"), options(tuple(1)));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 0);
+        checkRow(1, results, 0, 2, 0);
+
+        results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < ?"), options(tuple(2)));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 0);
+        checkRow(1, results, 0, 1, 0);
+
+        results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= ?"), options(tuple(1)));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 0);
+        checkRow(1, results, 0, 1, 0);
+
+
+        results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > ? AND (b) < ?"),
+                options(tuple(0), tuple(2)));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0);
+    }
+
+    @Test
+    public void testPrepareMultipleClusteringColumnInequality() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)");
+
+        UntypedResultSet results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (?)"), makeIntOptions(0));
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(2, results, 0, 1, 1, 1);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (?, ?)"), makeIntOptions(1, 0));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 1, 0);
+        checkRow(1, results, 0, 1, 1, 1);
+
+        results = executePrepared(prepare
+                ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?)"), makeIntOptions(1, 1, 0));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b) < (?)"),
+                makeIntOptions(0, 1, 0, 1));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 0, 1, 1);
+
+        results = executePrepared(prepare
+                ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)"),
+                makeIntOptions(0, 1, 1, 1, 1));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)"),
+                makeIntOptions(0, 1, 1, 1, 1, 0));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+
+        // reversed
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (?) ORDER BY b DESC, c DESC, d DESC"),
+                makeIntOptions(0));
+        assertEquals(3, results.size());
+        checkRow(2, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC"),
+                makeIntOptions(0, 1, 1, 1, 1));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+    }
+
+    @Test
+    public void testPrepareMultipleClusteringColumnInequalitySingleMarker() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)");
+
+        UntypedResultSet results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > ?"), options(tuple(0)));
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(2, results, 0, 1, 1, 1);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > ?"), options(tuple(1, 0)));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 1, 1, 0);
+        checkRow(1, results, 0, 1, 1, 1);
+
+        results = executePrepared(prepare
+                ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ?"), options(tuple(1, 1, 0)));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b) < ?"),
+                options(tuple(0, 1, 0), tuple(1)));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 0, 1, 1);
+
+        results = executePrepared(prepare
+                ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c) < ?"),
+                options(tuple(0, 1, 1), tuple(1, 1)));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c, d) < ?"),
+                options(tuple(0, 1, 1), tuple(1, 1, 0)));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+
+        // reversed
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > ? ORDER BY b DESC, c DESC, d DESC"),
+                options(tuple(0)));
+        assertEquals(3, results.size());
+        checkRow(2, results, 0, 1, 0, 0);
+        checkRow(1, results, 0, 1, 1, 0);
+        checkRow(0, results, 0, 1, 1, 1);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c) < ? ORDER BY b DESC, c DESC, d DESC"),
+                options(tuple(0, 1, 1), tuple(1, 1)));
+        assertEquals(1, results.size());
+        checkRow(0, results, 0, 1, 0, 0);
+    }
+
+    @Test
+    public void testPrepareLiteralIn() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+        UntypedResultSet results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))"),
+                makeIntOptions(0, 1, 0, 0, 1, 1));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        // same query, but reversed order for the IN values
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))"),
+                makeIntOptions(0, 1, 1, 0, 1, 0));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN ((?, ?))"),
+                makeIntOptions(0, 1));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ((?))"),
+                makeIntOptions(0));
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+    }
+
+    @Test
+    public void testPrepareInOneMarkerPerTuple() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+        UntypedResultSet results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN (?, ?)"),
+                options(tuple(0, 1, 0), tuple(0, 1, 1)));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        // same query, but reversed order for the IN values
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN (?, ?)"),
+                options(tuple(0, 1, 1), tuple(0, 1, 0)));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+
+        results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN (?)"),
+                options(tuple(0, 1)));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN (?)"),
+                options(tuple(0)));
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+    }
+
+    @Test
+    public void testPrepareInOneMarker() throws Throwable
+    {
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+        execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+        UntypedResultSet results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"),
+                options(list(tuple(0, 1, 0), tuple(0, 1, 1))));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        // same query, but reversed order for the IN values
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"),
+                options(list(tuple(0, 1, 1), tuple(0, 1, 0))));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        results = executePrepared(prepare(
+                "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"),
+                options(list()));
+        assertTrue(results.isEmpty());
+
+        results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN ?"),
+                options(list(tuple(0, 1))));
+        assertEquals(2, results.size());
+        checkRow(0, results, 0, 0, 1, 0);
+        checkRow(1, results, 0, 0, 1, 1);
+
+        results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ?"),
+                options(list(tuple(0))));
+        assertEquals(3, results.size());
+        checkRow(0, results, 0, 0, 0, 0);
+        checkRow(1, results, 0, 0, 1, 0);
+        checkRow(2, results, 0, 0, 1, 1);
+
+        results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ?"),
+                options(list()));
+        assertTrue(results.isEmpty());
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareLiteralInWithShortTuple() throws Throwable
+    {
+        prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?))");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareLiteralInWithLongTuple() throws Throwable
+    {
+        prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareLiteralInWithPartitionKey() throws Throwable
+    {
+        prepare("SELECT * FROM %s.multiple_clustering WHERE (a, b, c, d) IN ((?, ?, ?, ?))");
+    }
+
+    @Test(expected=InvalidRequestException.class)
+    public void testPrepareLiteralInSkipsClusteringColumn() throws Throwable
+    {
+        prepare("SELECT * FROM %s.multiple_clustering WHERE (c, d) IN ((?, ?))");
+    }
+
+    private static QueryOptions makeIntOptions(Integer... values)
+    {
+        List<ByteBuffer> buffers = new ArrayList<>(values.length);
+        for (int value : values)
+            buffers.add(ByteBufferUtil.bytes(value));
+        return new QueryOptions(ConsistencyLevel.ONE, buffers);
+    }
+
+    private static ByteBuffer tuple(Integer... values)
+    {
+        List<AbstractType<?>> types = new ArrayList<>(values.length);
+        ByteBuffer[] buffers = new ByteBuffer[values.length];
+        for (int i = 0; i < values.length; i++)
+        {
+            types.add(Int32Type.instance);
+            buffers[i] = ByteBufferUtil.bytes(values[i]);
+        }
+
+        TupleType type = new TupleType(types);
+        return type.buildValue(buffers);
+    }
+
+    private static ByteBuffer list(ByteBuffer... values)
+    {
+        return CollectionType.pack(Arrays.asList(values), values.length);
+    }
+
+    private static QueryOptions options(ByteBuffer... buffers)
+    {
+        return new QueryOptions(ConsistencyLevel.ONE, Arrays.asList(buffers));
+    }
+
+    private static void checkRow(int rowIndex, UntypedResultSet results, Integer... expectedValues)
+    {
+        List<UntypedResultSet.Row> rows = newArrayList(results.iterator());
+        UntypedResultSet.Row row = rows.get(rowIndex);
+        Iterator<ColumnSpecification> columns = row.getColumns().iterator();
+        for (Integer expected : expectedValues)
+        {
+            String columnName = columns.next().name.toString();
+            int actual = row.getInt(columnName);
+            assertEquals(String.format("Expected value %d for column %s in row %d, but got %s", actual, columnName, rowIndex, expected),
+                         (long) expected, actual);
+        }
+    }
+}
\ No newline at end of file


[3/7] git commit: Support multi-row selects within a partition using IN

Posted by ty...@apache.org.
Support multi-row selects within a partition using IN

Patch by Tyler Hobbs; review by Sylvain Lebresne for CASSANDRA-6875


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

Branch: refs/heads/trunk
Commit: 43496384d404f2fa0af943003f2dc8fdfced4073
Parents: 2635632
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Thu May 22 14:04:48 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Thu May 22 14:04:48 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |    2 +-
 .../apache/cassandra/cql/SelectStatement.java   |    1 -
 .../apache/cassandra/cql3/AbstractMarker.java   |   10 +-
 .../org/apache/cassandra/cql3/CQLStatement.java |    2 +-
 .../org/apache/cassandra/cql3/Constants.java    |    6 +
 src/java/org/apache/cassandra/cql3/Cql.g        |  107 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |   13 +-
 .../cassandra/cql3/MultiColumnRelation.java     |  144 +++
 .../apache/cassandra/cql3/QueryProcessor.java   |    2 +-
 .../org/apache/cassandra/cql3/Relation.java     |  107 +-
 .../org/apache/cassandra/cql3/ResultSet.java    |    1 -
 .../cassandra/cql3/SingleColumnRelation.java    |   95 ++
 src/java/org/apache/cassandra/cql3/Term.java    |   10 +
 src/java/org/apache/cassandra/cql3/Tuples.java  |  349 ++++++
 .../statements/AuthenticationStatement.java     |    2 +-
 .../cql3/statements/AuthorizationStatement.java |    2 +-
 .../cql3/statements/BatchStatement.java         |    4 +-
 .../cql3/statements/ModificationStatement.java  |   25 +-
 .../cql3/statements/MultiColumnRestriction.java |  135 +++
 .../cassandra/cql3/statements/Restriction.java  |  287 +----
 .../statements/SchemaAlteringStatement.java     |    2 +-
 .../cql3/statements/SelectStatement.java        |  829 ++++++++-----
 .../statements/SingleColumnRestriction.java     |  300 +++++
 .../cql3/statements/TruncateStatement.java      |    2 +-
 .../cassandra/cql3/statements/UseStatement.java |    2 +-
 .../apache/cassandra/db/marshal/TupleType.java  |  279 +++++
 .../cassandra/cql3/MultiColumnRelationTest.java | 1112 ++++++++++++++++++
 27 files changed, 3138 insertions(+), 692 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bddb1d1..c6c51c3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,6 +1,6 @@
 2.0.9
  * Add missing iso8601 patterns for date strings (6973)
-
+ * Support selecting multiple rows in a partition using IN (CASSANDRA-6875)
 
 2.0.8
  * Always reallocate buffers in HSHA (CASSANDRA-6285)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/SelectStatement.java b/src/java/org/apache/cassandra/cql/SelectStatement.java
index 1126738..7dd5592 100644
--- a/src/java/org/apache/cassandra/cql/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql/SelectStatement.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.cql;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Set;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/AbstractMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/AbstractMarker.java b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
index 165cb00..4329ed9 100644
--- a/src/java/org/apache/cassandra/cql3/AbstractMarker.java
+++ b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
@@ -21,7 +21,6 @@ import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
-
 /**
  * A single bind marker.
  */
@@ -84,7 +83,12 @@ public abstract class AbstractMarker extends Term.NonTerminal
         }
     }
 
-    // A raw that stands for multiple values, i.e. when we have 'IN ?'
+    /**
+     * A raw placeholder for multiple values of the same type for a single column.
+     * For example, "SELECT ... WHERE user_id IN ?'.
+     *
+     * Because a single type is used, a List is used to represent the values.
+     */
     public static class INRaw extends Raw
     {
         public INRaw(int bindIndex)
@@ -102,7 +106,7 @@ public abstract class AbstractMarker extends Term.NonTerminal
         public AbstractMarker prepare(ColumnSpecification receiver) throws InvalidRequestException
         {
             if (receiver.type instanceof CollectionType)
-                throw new InvalidRequestException("Invalid IN relation on collection column");
+                throw new InvalidRequestException("Collection columns do not support IN relations");
 
             return new Lists.Marker(bindIndex, makeInReceiver(receiver));
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/CQLStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQLStatement.java b/src/java/org/apache/cassandra/cql3/CQLStatement.java
index 81cd2b2..a1642ef 100644
--- a/src/java/org/apache/cassandra/cql3/CQLStatement.java
+++ b/src/java/org/apache/cassandra/cql3/CQLStatement.java
@@ -57,5 +57,5 @@ public interface CQLStatement
      *
      * @param state the current query state
      */
-    public ResultMessage executeInternal(QueryState state) throws RequestValidationException, RequestExecutionException;
+    public ResultMessage executeInternal(QueryState state, QueryOptions options) throws RequestValidationException, RequestExecutionException;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index f99fd02..4ea6c2d 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -254,6 +254,12 @@ public abstract class Constants
         {
             return bytes;
         }
+
+        @Override
+        public String toString()
+        {
+            return ByteBufferUtil.bytesToHex(bytes);
+        }
     }
 
     public static class Marker extends AbstractMarker

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 04f9f59..ceb2bde 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -67,6 +67,20 @@ options {
         return marker;
     }
 
+    public Tuples.Raw newTupleBindVariables(ColumnIdentifier name)
+    {
+        Tuples.Raw marker = new Tuples.Raw(bindVariables.size());
+        bindVariables.add(name);
+        return marker;
+    }
+
+    public Tuples.INRaw newTupleINBindVariables(ColumnIdentifier name)
+    {
+        Tuples.INRaw marker = new Tuples.INRaw(bindVariables.size());
+        bindVariables.add(name);
+        return marker;
+    }
+
     public void displayRecognitionError(String[] tokenNames, RecognitionException e)
     {
         String hdr = getErrorHeader(e);
@@ -880,38 +894,79 @@ relationType returns [Relation.Type op]
     ;
 
 relation[List<Relation> clauses]
-    : name=cident type=relationType t=term { $clauses.add(new Relation(name, type, t)); }
-    | K_TOKEN 
-        { List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>(); }
-          '(' name1=cident { l.add(name1); } ( ',' namen=cident { l.add(namen); })* ')'
-        type=relationType t=term
+    : name=cident type=relationType t=term { $clauses.add(new SingleColumnRelation(name, type, t)); }
+    | K_TOKEN l=tupleOfIdentifiers type=relationType t=term
         {
             for (ColumnIdentifier id : l)
-                $clauses.add(new Relation(id, type, t, true));
+                $clauses.add(new SingleColumnRelation(id, type, t, true));
         }
-    | name=cident K_IN { Term.Raw marker = null; } (QMARK { marker = newINBindVariables(null); } | ':' mid=cident { marker = newINBindVariables(mid); })
-        { $clauses.add(new Relation(name, Relation.Type.IN, marker)); }
-    | name=cident K_IN { Relation rel = Relation.createInRelation($name.id); }
-       '(' ( f1=term { rel.addInValue(f1); } (',' fN=term { rel.addInValue(fN); } )* )? ')' { $clauses.add(rel); }
-    | {
-         List<ColumnIdentifier> ids = new ArrayList<ColumnIdentifier>();
-         List<Term.Raw> terms = new ArrayList<Term.Raw>();
-      }
-        '(' n1=cident { ids.add(n1); } (',' ni=cident { ids.add(ni); })* ')'
-        type=relationType
-        '(' t1=term { terms.add(t1); } (',' ti=term { terms.add(ti); })* ')'
-      {
-          if (type == Relation.Type.IN)
-              addRecognitionError("Cannot use IN relation with tuple notation");
-          if (ids.size() != terms.size())
-              addRecognitionError(String.format("Number of values (" + terms.size() + ") in tuple notation doesn't match the number of column names (" + ids.size() + ")"));
-          else
-              for (int i = 0; i < ids.size(); i++)
-                  $clauses.add(new Relation(ids.get(i), type, terms.get(i), i == 0 ? null : ids.get(i-1)));
-      }
+    | name=cident K_IN marker=inMarker
+        { $clauses.add(new SingleColumnRelation(name, Relation.Type.IN, marker)); }
+    | name=cident K_IN inValues=singleColumnInValues
+        { $clauses.add(SingleColumnRelation.createInRelation($name.id, inValues)); }
+    | ids=tupleOfIdentifiers
+      ( K_IN
+          ( '(' ')'
+              { $clauses.add(MultiColumnRelation.createInRelation(ids, new ArrayList<Tuples.Literal>())); }
+          | tupleInMarker=inMarkerForTuple /* (a, b, c) IN ? */
+              { $clauses.add(MultiColumnRelation.createSingleMarkerInRelation(ids, tupleInMarker)); }
+          | literals=tupleOfTupleLiterals /* (a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) */
+              {
+                  $clauses.add(MultiColumnRelation.createInRelation(ids, literals));
+              }
+          | markers=tupleOfMarkersForTuples /* (a, b, c) IN (?, ?, ...) */
+              { $clauses.add(MultiColumnRelation.createInRelation(ids, markers)); }
+          )
+      | type=relationType literal=tupleLiteral /* (a, b, c) > (1, 2, 3) or (a, b, c) > (?, ?, ?) */
+          {
+              $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, literal));
+          }
+      | type=relationType tupleMarker=markerForTuple /* (a, b, c) >= ? */
+          { $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, tupleMarker)); }
+      )
     | '(' relation[$clauses] ')'
     ;
 
+inMarker returns [AbstractMarker.INRaw marker]
+    : QMARK { $marker = newINBindVariables(null); }
+    | ':' name=cident { $marker = newINBindVariables(name); }
+    ;
+
+tupleOfIdentifiers returns [List<ColumnIdentifier> ids]
+    @init { $ids = new ArrayList<ColumnIdentifier>(); }
+    : '(' n1=cident { $ids.add(n1); } (',' ni=cident { $ids.add(ni); })* ')'
+    ;
+
+singleColumnInValues returns [List<Term.Raw> terms]
+    @init { $terms = new ArrayList<Term.Raw>(); }
+    : '(' ( t1 = term { $terms.add(t1); } (',' ti=term { $terms.add(ti); })* )? ')'
+    ;
+
+tupleLiteral returns [Tuples.Literal literal]
+    @init { List<Term.Raw> terms = new ArrayList<>(); }
+    : '(' t1=term { terms.add(t1); } (',' ti=term { terms.add(ti); })* ')' { $literal = new Tuples.Literal(terms); }
+    ;
+
+tupleOfTupleLiterals returns [List<Tuples.Literal> literals]
+    @init { $literals = new ArrayList<>(); }
+    : '(' t1=tupleLiteral { $literals.add(t1); } (',' ti=tupleLiteral { $literals.add(ti); })* ')'
+    ;
+
+markerForTuple returns [Tuples.Raw marker]
+    : QMARK { $marker = newTupleBindVariables(null); }
+    | ':' name=cident { $marker = newTupleBindVariables(name); }
+    ;
+
+tupleOfMarkersForTuples returns [List<Tuples.Raw> markers]
+    @init { $markers = new ArrayList<Tuples.Raw>(); }
+    : '(' m1=markerForTuple { $markers.add(m1); } (',' mi=markerForTuple { $markers.add(mi); })* ')'
+    ;
+
+inMarkerForTuple returns [Tuples.INRaw marker]
+    : QMARK { $marker = newTupleINBindVariables(null); }
+    | ':' name=cident { $marker = newTupleINBindVariables(name); }
+    ;
+
 comparatorType returns [CQL3Type t]
     : c=native_type     { $t = c; }
     | c=collection_type { $t = c; }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index 4ad39db..d483dd5 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -117,7 +117,7 @@ public abstract class Lists
         }
     }
 
-    public static class Value extends Term.Terminal
+    public static class Value extends Term.MultiItemTerminal
     {
         public final List<ByteBuffer> elements;
 
@@ -148,9 +148,14 @@ public abstract class Lists
         {
             return CollectionType.pack(elements, elements.size());
         }
+
+        public List<ByteBuffer> getElements()
+        {
+            return elements;
+        }
     }
 
-    /*
+    /**
      * Basically similar to a Value, but with some non-pure function (that need
      * to be evaluated at execution time) in it.
      *
@@ -200,6 +205,9 @@ public abstract class Lists
         }
     }
 
+    /**
+     * A marker for List values and IN relations
+     */
     public static class Marker extends AbstractMarker
     {
         protected Marker(int bindIndex, ColumnSpecification receiver)
@@ -212,7 +220,6 @@ public abstract class Lists
         {
             ByteBuffer value = values.get(bindIndex);
             return value == null ? null : Value.fromSerialized(value, (ListType)receiver.type);
-
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
new file mode 100644
index 0000000..fda60df
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -0,0 +1,144 @@
+/*
+ * 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.List;
+
+/**
+ * A relation using the tuple notation, which typically affects multiple columns.
+ * Examples:
+ *  - SELECT ... WHERE (a, b, c) > (1, 'a', 10)
+ *  - SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))
+ *  - SELECT ... WHERE (a, b) < ?
+ *  - SELECT ... WHERE (a, b) IN ?
+ */
+public class MultiColumnRelation extends Relation
+{
+    private final List<ColumnIdentifier> entities;
+
+    /** A Tuples.Literal or Tuples.Raw marker */
+    private final Term.MultiColumnRaw valuesOrMarker;
+
+    /** A list of Tuples.Literal or Tuples.Raw markers */
+    private final List<? extends Term.MultiColumnRaw> inValues;
+
+    private final Tuples.INRaw inMarker;
+
+    private MultiColumnRelation(List<ColumnIdentifier> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
+    {
+        this.entities = entities;
+        this.relationType = relationType;
+        this.valuesOrMarker = valuesOrMarker;
+
+        this.inValues = inValues;
+        this.inMarker = inMarker;
+    }
+
+    /**
+     * Creates a multi-column EQ, LT, LTE, GT, or GTE relation.
+     * For example: "SELECT ... WHERE (a, b) > (0, 1)"
+     * @param entities the columns on the LHS of the relation
+     * @param relationType the relation operator
+     * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker
+     */
+    public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker)
+    {
+        assert relationType != Relation.Type.IN;
+        return new MultiColumnRelation(entities, relationType, valuesOrMarker, null, null);
+    }
+
+    /**
+     * Creates a multi-column IN relation with a list of IN values or markers.
+     * For example: "SELECT ... WHERE (a, b) IN ((0, 1), (2, 3))"
+     * @param entities the columns on the LHS of the relation
+     * @param inValues a list of Tuples.Literal instances or a Tuples.Raw markers
+     */
+    public static MultiColumnRelation createInRelation(List<ColumnIdentifier> entities, List<? extends Term.MultiColumnRaw> inValues)
+    {
+        return new MultiColumnRelation(entities, Relation.Type.IN, null, inValues, null);
+    }
+
+    /**
+     * Creates a multi-column IN relation with a marker for the IN values.
+     * For example: "SELECT ... WHERE (a, b) IN ?"
+     * @param entities the columns on the LHS of the relation
+     * @param inMarker a single IN marker
+     */
+    public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier> entities, Tuples.INRaw inMarker)
+    {
+        return new MultiColumnRelation(entities, Relation.Type.IN, null, null, inMarker);
+    }
+
+    public List<ColumnIdentifier> getEntities()
+    {
+        return entities;
+    }
+
+    /**
+     * For non-IN relations, returns the Tuples.Literal or Tuples.Raw marker for a single tuple.
+     */
+    public Term.MultiColumnRaw getValue()
+    {
+        assert relationType != Relation.Type.IN;
+        return valuesOrMarker;
+    }
+
+    /**
+     * For IN relations, returns the list of Tuples.Literal instances or Tuples.Raw markers.
+     * If a single IN marker was used, this will return null;
+     */
+    public List<? extends Term.MultiColumnRaw> getInValues()
+    {
+
+        return inValues;
+    }
+
+    /**
+     * For IN relations, returns the single marker for the IN values if there is one, otherwise null.
+     */
+    public Tuples.INRaw getInMarker()
+    {
+        return inMarker;
+    }
+
+    public boolean isMultiColumn()
+    {
+        return true;
+    }
+
+    @Override
+    public String toString()
+    {
+        if (relationType == Type.IN)
+        {
+            StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
+            sb.append(" IN ");
+            sb.append(inMarker != null ? '?' : Tuples.tupleToString(inValues));
+            return sb.toString();
+        }
+        else
+        {
+            StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
+            sb.append(" ");
+            sb.append(relationType);
+            sb.append(" ");
+            sb.append(valuesOrMarker);
+            return sb.toString();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 15ee59f..30d1bd7 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -205,7 +205,7 @@ public class QueryProcessor implements QueryHandler
             state.setKeyspace(Keyspace.SYSTEM_KS);
             CQLStatement statement = getStatement(query, state).statement;
             statement.validate(state);
-            ResultMessage result = statement.executeInternal(qState);
+            ResultMessage result = statement.executeInternal(qState, QueryOptions.DEFAULT);
             if (result instanceof ResultMessage.Rows)
                 return new UntypedResultSet(((ResultMessage.Rows)result).result);
             else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index 9d065bf..0f1366d 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -17,65 +17,35 @@
  */
 package org.apache.cassandra.cql3;
 
-import java.util.ArrayList;
-import java.util.List;
+public abstract class Relation {
 
-/**
- * Relations encapsulate the relationship between an entity of some kind, and
- * a value (term). For example, <key> > "start" or "colname1" = "somevalue".
- *
- */
-public class Relation
-{
-    private final ColumnIdentifier entity;
-    private final Type relationType;
-    private final Term.Raw value;
-    private final List<Term.Raw> inValues;
-    public final boolean onToken;
-
-    // Will be null unless for tuple notations (#4851)
-    public final ColumnIdentifier previousInTuple;
+    protected Type relationType;
 
     public static enum Type
     {
         EQ, LT, LTE, GTE, GT, IN;
-    }
-
-    private Relation(ColumnIdentifier entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken, ColumnIdentifier previousInTuple)
-    {
-        this.entity = entity;
-        this.relationType = type;
-        this.value = value;
-        this.inValues = inValues;
-        this.onToken = onToken;
-        this.previousInTuple = previousInTuple;
-    }
-
-    /**
-     * Creates a new relation.
-     *
-     * @param entity the kind of relation this is; what the term is being compared to.
-     * @param type the type that describes how this entity relates to the value.
-     * @param value the value being compared.
-     */
-    public Relation(ColumnIdentifier entity, Type type, Term.Raw value)
-    {
-        this(entity, type, value, null, false, null);
-    }
-
-    public Relation(ColumnIdentifier entity, Type type, Term.Raw value, boolean onToken)
-    {
-        this(entity, type, value, null, onToken, null);
-    }
 
-    public Relation(ColumnIdentifier entity, Type type, Term.Raw value, ColumnIdentifier previousInTuple)
-    {
-        this(entity, type, value, null, false, previousInTuple);
-    }
-
-    public static Relation createInRelation(ColumnIdentifier entity)
-    {
-        return new Relation(entity, Type.IN, null, new ArrayList<Term.Raw>(), false, null);
+        @Override
+        public String toString()
+        {
+            switch (this)
+            {
+                case EQ:
+                    return "=";
+                case LT:
+                    return "<";
+                case LTE:
+                    return "<=";
+                case GT:
+                    return ">";
+                case GTE:
+                    return ">=";
+                case IN:
+                    return "IN";
+                default:
+                    return this.name();
+            }
+        }
     }
 
     public Type operator()
@@ -83,34 +53,5 @@ public class Relation
         return relationType;
     }
 
-    public ColumnIdentifier getEntity()
-    {
-        return entity;
-    }
-
-    public Term.Raw getValue()
-    {
-        assert relationType != Type.IN || value == null || value instanceof AbstractMarker.INRaw;
-        return value;
-    }
-
-    public List<Term.Raw> getInValues()
-    {
-        assert relationType == Type.IN;
-        return inValues;
-    }
-
-    public void addInValue(Term.Raw t)
-    {
-        inValues.add(t);
-    }
-
-    @Override
-    public String toString()
-    {
-        if (relationType == Type.IN)
-            return String.format("%s IN %s", entity, inValues);
-        else
-            return String.format("%s %s %s", entity, relationType, value);
-    }
+    public abstract boolean isMultiColumn();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index e4f27f9..4cda0cd 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
 
 import org.apache.cassandra.transport.*;
 import org.apache.cassandra.db.marshal.AbstractType;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
new file mode 100644
index 0000000..5464c23
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.util.List;
+
+/**
+ * Relations encapsulate the relationship between an entity of some kind, and
+ * a value (term). For example, <key> > "start" or "colname1" = "somevalue".
+ *
+ */
+public class SingleColumnRelation extends Relation
+{
+    private final ColumnIdentifier entity;
+    private final Term.Raw value;
+    private final List<Term.Raw> inValues;
+    public final boolean onToken;
+
+    private SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
+    {
+        this.entity = entity;
+        this.relationType = type;
+        this.value = value;
+        this.inValues = inValues;
+        this.onToken = onToken;
+    }
+
+    /**
+     * Creates a new relation.
+     *
+     * @param entity the kind of relation this is; what the term is being compared to.
+     * @param type the type that describes how this entity relates to the value.
+     * @param value the value being compared.
+     */
+    public SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value)
+    {
+        this(entity, type, value, null, false);
+    }
+
+    public SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value, boolean onToken)
+    {
+        this(entity, type, value, null, onToken);
+    }
+
+    public static SingleColumnRelation createInRelation(ColumnIdentifier entity, List<Term.Raw> inValues)
+    {
+        return new SingleColumnRelation(entity, Type.IN, null, inValues, false);
+    }
+
+    public ColumnIdentifier getEntity()
+    {
+        return entity;
+    }
+
+    public Term.Raw getValue()
+    {
+        assert relationType != Type.IN || value == null || value instanceof AbstractMarker.INRaw;
+        return value;
+    }
+
+    public List<Term.Raw> getInValues()
+    {
+        assert relationType == Type.IN;
+        return inValues;
+    }
+
+    public boolean isMultiColumn()
+    {
+        return false;
+    }
+
+    @Override
+    public String toString()
+    {
+        if (relationType == Type.IN)
+            return String.format("%s IN %s", entity, inValues);
+        else
+            return String.format("%s %s %s", entity, relationType, value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/Term.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Term.java b/src/java/org/apache/cassandra/cql3/Term.java
index d69fc33..96b4b71 100644
--- a/src/java/org/apache/cassandra/cql3/Term.java
+++ b/src/java/org/apache/cassandra/cql3/Term.java
@@ -91,6 +91,11 @@ public interface Term
         public Term prepare(ColumnSpecification receiver) throws InvalidRequestException;
     }
 
+    public interface MultiColumnRaw extends Raw
+    {
+        public Term prepare(List<? extends ColumnSpecification> receiver) throws InvalidRequestException;
+    }
+
     /**
      * A terminal term, one that can be reduced to a byte buffer directly.
      *
@@ -128,6 +133,11 @@ public interface Term
         }
     }
 
+    public abstract class MultiItemTerminal extends Terminal
+    {
+        public abstract List<ByteBuffer> getElements();
+    }
+
     /**
      * A non terminal term, i.e. a term that can only be reduce to a byte buffer
      * at execution time.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
new file mode 100644
index 0000000..9e86912
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -0,0 +1,349 @@
+/*
+ * 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 org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+/**
+ * Static helper methods and classes for tuples.
+ */
+public class Tuples
+{
+    private static final Logger logger = LoggerFactory.getLogger(Tuples.class);
+
+    /**
+     * A raw, literal tuple.  When prepared, this will become a Tuples.Value or Tuples.DelayedValue, depending
+     * on whether the tuple holds NonTerminals.
+     */
+    public static class Literal implements Term.MultiColumnRaw
+    {
+        private final List<Term.Raw> elements;
+
+        public Literal(List<Term.Raw> elements)
+        {
+            this.elements = elements;
+        }
+
+        public Term prepare(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+        {
+            if (elements.size() != receivers.size())
+                throw new InvalidRequestException(String.format("Expected %d elements in value tuple, but got %d: %s", receivers.size(), elements.size(), this));
+
+            List<Term> values = new ArrayList<>(elements.size());
+            boolean allTerminal = true;
+            for (int i = 0; i < elements.size(); i++)
+            {
+                Term t = elements.get(i).prepare(receivers.get(i));
+                if (t instanceof Term.NonTerminal)
+                    allTerminal = false;
+
+                values.add(t);
+            }
+            DelayedValue value = new DelayedValue(values);
+            return allTerminal ? value.bind(Collections.<ByteBuffer>emptyList()) : value;
+        }
+
+        public Term prepare(ColumnSpecification receiver)
+        {
+            throw new AssertionError("Tuples.Literal instances require a list of receivers for prepare()");
+        }
+
+        public boolean isAssignableTo(ColumnSpecification receiver)
+        {
+            // tuples shouldn't be assignable to anything right now
+            return false;
+        }
+
+        @Override
+        public String toString()
+        {
+            return tupleToString(elements);
+        }
+    }
+
+    /**
+     * A tuple of terminal values (e.g (123, 'abc')).
+     */
+    public static class Value extends Term.MultiItemTerminal
+    {
+        public final ByteBuffer[] elements;
+
+        public Value(ByteBuffer[] elements)
+        {
+            this.elements = elements;
+        }
+
+        public static Value fromSerialized(ByteBuffer bytes, TupleType type)
+        {
+            return new Value(type.split(bytes));
+        }
+
+        public ByteBuffer get()
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public List<ByteBuffer> getElements()
+        {
+            return Arrays.asList(elements);
+        }
+    }
+
+    /**
+     * Similar to Value, but contains at least one NonTerminal, such as a non-pure functions or bind marker.
+     */
+    public static class DelayedValue extends Term.NonTerminal
+    {
+        public final List<Term> elements;
+
+        public DelayedValue(List<Term> elements)
+        {
+            this.elements = elements;
+        }
+
+        public boolean containsBindMarker()
+        {
+            for (Term term : elements)
+                if (term.containsBindMarker())
+                    return true;
+
+            return false;
+        }
+
+        public void collectMarkerSpecification(VariableSpecifications boundNames)
+        {
+            for (Term term : elements)
+                term.collectMarkerSpecification(boundNames);
+        }
+
+        public Value bind(List<ByteBuffer> values) throws InvalidRequestException
+        {
+            ByteBuffer[] buffers = new ByteBuffer[elements.size()];
+            for (int i=0; i < elements.size(); i++)
+            {
+                ByteBuffer bytes = elements.get(i).bindAndGet(values);
+                if (bytes == null)
+                    throw new InvalidRequestException("Tuples may not contain null values");
+
+                buffers[i] = elements.get(i).bindAndGet(values);
+            }
+            return new Value(buffers);
+        }
+
+        @Override
+        public String toString()
+        {
+            return tupleToString(elements);
+        }
+    }
+
+    /**
+     * A terminal value for a list of IN values that are tuples. For example: "SELECT ... WHERE (a, b, c) IN ?"
+     * This is similar to Lists.Value, but allows us to keep components of the tuples in the list separate.
+     */
+    public static class InValue extends Term.Terminal
+    {
+        List<List<ByteBuffer>> elements;
+
+        public InValue(List<List<ByteBuffer>> items)
+        {
+            this.elements = items;
+        }
+
+        public static InValue fromSerialized(ByteBuffer value, ListType type) throws InvalidRequestException
+        {
+            try
+            {
+                // Collections have this small hack that validate cannot be called on a serialized object,
+                // but compose does the validation (so we're fine).
+                List<?> l = (List<?>)type.compose(value);
+
+                assert type.elements instanceof TupleType;
+                TupleType tupleType = (TupleType) type.elements;
+
+                // type.split(bytes)
+                List<List<ByteBuffer>> elements = new ArrayList<>(l.size());
+                for (Object element : l)
+                    elements.add(Arrays.asList(tupleType.split(type.elements.decompose(element))));
+                return new InValue(elements);
+            }
+            catch (MarshalException e)
+            {
+                throw new InvalidRequestException(e.getMessage());
+            }
+        }
+
+        public ByteBuffer get()
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public List<List<ByteBuffer>> getSplitValues()
+        {
+            return elements;
+        }
+    }
+
+    /**
+     * A raw placeholder for a tuple of values for different multiple columns, each of which may have a different type.
+     * For example, "SELECT ... WHERE (col1, col2) > ?".
+     */
+    public static class Raw extends AbstractMarker.Raw implements Term.MultiColumnRaw
+    {
+        public Raw(int bindIndex)
+        {
+            super(bindIndex);
+        }
+
+        private static ColumnSpecification makeReceiver(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+        {
+            List<AbstractType<?>> types = new ArrayList<>(receivers.size());
+            StringBuilder inName = new StringBuilder("(");
+            for (int i = 0; i < receivers.size(); i++)
+            {
+                ColumnSpecification receiver = receivers.get(i);
+                inName.append(receiver.name);
+                if (i < receivers.size() - 1)
+                    inName.append(",");
+                types.add(receiver.type);
+            }
+            inName.append(')');
+
+            ColumnIdentifier identifier = new ColumnIdentifier(inName.toString(), true);
+            TupleType type = new TupleType(types);
+            return new ColumnSpecification(receivers.get(0).ksName, receivers.get(0).cfName, identifier, type);
+        }
+
+        public AbstractMarker prepare(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+        {
+            return new Tuples.Marker(bindIndex, makeReceiver(receivers));
+        }
+
+        @Override
+        public AbstractMarker prepare(ColumnSpecification receiver)
+        {
+            throw new AssertionError("Tuples.Raw.prepare() requires a list of receivers");
+        }
+    }
+
+    /**
+     * A raw marker for an IN list of tuples, like "SELECT ... WHERE (a, b, c) IN ?"
+     */
+    public static class INRaw extends AbstractMarker.Raw
+    {
+        public INRaw(int bindIndex)
+        {
+            super(bindIndex);
+        }
+
+        private static ColumnSpecification makeInReceiver(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+        {
+            List<AbstractType<?>> types = new ArrayList<>(receivers.size());
+            StringBuilder inName = new StringBuilder("in(");
+            for (int i = 0; i < receivers.size(); i++)
+            {
+                ColumnSpecification receiver = receivers.get(i);
+                inName.append(receiver.name);
+                if (i < receivers.size() - 1)
+                    inName.append(",");
+
+                if (receiver.type instanceof CollectionType)
+                    throw new InvalidRequestException("Collection columns do not support IN relations");
+                types.add(receiver.type);
+            }
+            inName.append(')');
+
+            ColumnIdentifier identifier = new ColumnIdentifier(inName.toString(), true);
+            TupleType type = new TupleType(types);
+            return new ColumnSpecification(receivers.get(0).ksName, receivers.get(0).cfName, identifier, ListType.getInstance(type));
+        }
+
+        public AbstractMarker prepare(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+        {
+            return new InMarker(bindIndex, makeInReceiver(receivers));
+        }
+
+        @Override
+        public AbstractMarker prepare(ColumnSpecification receiver)
+        {
+            throw new AssertionError("Tuples.INRaw.prepare() requires a list of receivers");
+        }
+    }
+
+    /**
+     * Represents a marker for a single tuple, like "SELECT ... WHERE (a, b, c) > ?"
+     */
+    public static class Marker extends AbstractMarker
+    {
+        public Marker(int bindIndex, ColumnSpecification receiver)
+        {
+            super(bindIndex, receiver);
+        }
+
+        public Value bind(List<ByteBuffer> values) throws InvalidRequestException
+        {
+            ByteBuffer value = values.get(bindIndex);
+            if (value == null)
+                return null;
+
+            return value == null ? null : Value.fromSerialized(value, (TupleType)receiver.type);
+        }
+    }
+
+    /**
+     * Represents a marker for a set of IN values that are tuples, like "SELECT ... WHERE (a, b, c) IN ?"
+     */
+    public static class InMarker extends AbstractMarker
+    {
+        protected InMarker(int bindIndex, ColumnSpecification receiver)
+        {
+            super(bindIndex, receiver);
+            assert receiver.type instanceof ListType;
+        }
+
+        public InValue bind(List<ByteBuffer> values) throws InvalidRequestException
+        {
+            ByteBuffer value = values.get(bindIndex);
+            return value == null ? null : InValue.fromSerialized(value, (ListType)receiver.type);
+        }
+    }
+
+    public static String tupleToString(List<?> items)
+    {
+
+        StringBuilder sb = new StringBuilder("(");
+        for (int i = 0; i < items.size(); i++)
+        {
+            sb.append(items.get(i));
+            if (i < items.size() - 1)
+                sb.append(", ");
+        }
+        sb.append(')');
+        return sb.toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
index 5fcf085..b47dd92 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
@@ -45,7 +45,7 @@ public abstract class AuthenticationStatement extends ParsedStatement implements
 
     public abstract ResultMessage execute(ClientState state) throws RequestExecutionException, RequestValidationException;
 
-    public ResultMessage executeInternal(QueryState state)
+    public ResultMessage executeInternal(QueryState state, QueryOptions options)
     {
         // executeInternal is for local query only, thus altering users doesn't make sense and is not supported
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
index db4581e..2c7f2cb 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
@@ -47,7 +47,7 @@ public abstract class AuthorizationStatement extends ParsedStatement implements
 
     public abstract ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException;
 
-    public ResultMessage executeInternal(QueryState state)
+    public ResultMessage executeInternal(QueryState state, QueryOptions options)
     {
         // executeInternal is for local query only, thus altering permission doesn't make sense and is not supported
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 6a1201b..875e41c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -327,11 +327,11 @@ public class BatchStatement implements CQLStatement, MeasurableForPreparedCache
         return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(ksName, key, cfName, result, columnsWithConditions, true));
     }
 
-    public ResultMessage executeInternal(QueryState queryState) throws RequestValidationException, RequestExecutionException
+    public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
     {
         assert !hasConditions;
 
-        for (IMutation mutation : getMutations(new PreparedBatchVariables(Collections.<ByteBuffer>emptyList()), true, null, queryState.getTimestamp()))
+        for (IMutation mutation : getMutations(new PreparedBatchVariables(options.getValues()), true, null, queryState.getTimestamp()))
             mutation.apply();
         return null;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 448722e..11aa0b1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -32,14 +32,11 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.db.marshal.BooleanType;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.service.CASConditions;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
@@ -250,14 +247,21 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
 
     public void addKeyValue(CFDefinition.Name name, Term value) throws InvalidRequestException
     {
-        addKeyValues(name, new Restriction.EQ(value, false));
+        addKeyValues(name, new SingleColumnRestriction.EQ(value, false));
     }
 
     public void processWhereClause(List<Relation> whereClause, VariableSpecifications names) throws InvalidRequestException
     {
         CFDefinition cfDef = cfm.getCfDef();
-        for (Relation rel : whereClause)
+        for (Relation relation : whereClause)
         {
+            if (!(relation instanceof SingleColumnRelation))
+            {
+                throw new InvalidRequestException(
+                        String.format("Multi-column relations cannot be used in WHERE clauses for modification statements: %s", relation));
+            }
+            SingleColumnRelation rel = (SingleColumnRelation) relation;
+
             CFDefinition.Name name = cfDef.get(rel.getEntity());
             if (name == null)
                 throw new InvalidRequestException(String.format("Unknown key identifier %s", rel.getEntity()));
@@ -272,7 +276,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                     {
                         Term t = rel.getValue().prepare(name);
                         t.collectMarkerSpecification(names);
-                        restriction = new Restriction.EQ(t, false);
+                        restriction = new SingleColumnRestriction.EQ(t, false);
                     }
                     else if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS && rel.operator() == Relation.Type.IN)
                     {
@@ -280,7 +284,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                         {
                             Term t = rel.getValue().prepare(name);
                             t.collectMarkerSpecification(names);
-                            restriction = Restriction.IN.create(t);
+                            restriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
                         }
                         else
                         {
@@ -291,7 +295,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                                 t.collectMarkerSpecification(names);
                                 values.add(t);
                             }
-                            restriction = Restriction.IN.create(values);
+                            restriction = new SingleColumnRestriction.InWithValues(values);
                         }
                     }
                     else
@@ -671,12 +675,13 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         return builder.build();
     }
 
-    public ResultMessage executeInternal(QueryState queryState) throws RequestValidationException, RequestExecutionException
+    public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
     {
         if (hasConditions())
             throw new UnsupportedOperationException();
 
-        for (IMutation mutation : getMutations(Collections.<ByteBuffer>emptyList(), true, null, queryState.getTimestamp()))
+        List<ByteBuffer> variables = options.getValues();
+        for (IMutation mutation : getMutations(variables, true, null, queryState.getTimestamp()))
             mutation.apply();
         return null;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
new file mode 100644
index 0000000..f643684
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
@@ -0,0 +1,135 @@
+/*
+ * 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.statements;
+
+import org.apache.cassandra.cql3.AbstractMarker;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public interface MultiColumnRestriction extends Restriction
+{
+    public static class EQ extends SingleColumnRestriction.EQ implements MultiColumnRestriction
+    {
+        public EQ(Term value, boolean onToken)
+        {
+            super(value, onToken);
+        }
+
+        public boolean isMultiColumn()
+        {
+            return true;
+        }
+
+        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            Tuples.Value t = (Tuples.Value)value.bind(variables);
+            return t.getElements();
+        }
+    }
+
+    public interface IN extends MultiColumnRestriction
+    {
+        public List<List<ByteBuffer>> splitValues(List<ByteBuffer> variables) throws InvalidRequestException;
+    }
+
+    /**
+     * An IN restriction that has a set of terms for in values.
+     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
+     */
+    public static class InWithValues extends SingleColumnRestriction.InWithValues implements MultiColumnRestriction.IN
+    {
+        public InWithValues(List<Term> values)
+        {
+            super(values);
+        }
+
+        public boolean isMultiColumn()
+        {
+            return true;
+        }
+
+        public List<List<ByteBuffer>> splitValues(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
+            for (Term value : values)
+            {
+                Term.MultiItemTerminal term = (Term.MultiItemTerminal)value.bind(variables);
+                buffers.add(term.getElements());
+            }
+            return buffers;
+        }
+    }
+
+    /**
+     * An IN restriction that uses a single marker for a set of IN values that are tuples.
+     * For example: "SELECT ... WHERE (a, b, c) IN ?"
+     */
+    public static class InWithMarker extends SingleColumnRestriction.InWithMarker implements MultiColumnRestriction.IN
+    {
+        public InWithMarker(AbstractMarker marker)
+        {
+            super(marker);
+        }
+
+        public boolean isMultiColumn()
+        {
+            return true;
+        }
+
+        public List<List<ByteBuffer>> splitValues(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            Tuples.InValue inValue = ((Tuples.InMarker) marker).bind(variables);
+            if (inValue == null)
+                throw new InvalidRequestException("Invalid null value for IN restriction");
+            return inValue.getSplitValues();
+        }
+    }
+
+    public static class Slice extends SingleColumnRestriction.Slice implements MultiColumnRestriction
+    {
+        public Slice(boolean onToken)
+        {
+            super(onToken);
+        }
+
+        public boolean isMultiColumn()
+        {
+            return true;
+        }
+
+        public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            throw new UnsupportedOperationException("Multicolumn slice restrictions do not support bound()");
+        }
+
+        /**
+         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
+         * ByteBuffer to represent the entire bound.
+         */
+        public List<ByteBuffer> componentBounds(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            Tuples.Value value = (Tuples.Value)bounds[b.idx].bind(variables);
+            return value.getElements();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/Restriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Restriction.java b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
index 6323acb..3d33bde 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@ -18,12 +18,8 @@
 package org.apache.cassandra.cql3.statements;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import com.google.common.base.Objects;
-
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.thrift.IndexOperator;
 import org.apache.cassandra.cql3.*;
@@ -39,289 +35,34 @@ public interface Restriction
     public boolean isSlice();
     public boolean isEQ();
     public boolean isIN();
+    public boolean isMultiColumn();
 
     // Only supported for EQ and IN, but it's convenient to have here
     public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException;
 
-    public static class EQ implements Restriction
-    {
-        private final Term value;
-        private final boolean onToken;
-
-        public EQ(Term value, boolean onToken)
-        {
-            this.value = value;
-            this.onToken = onToken;
-        }
-
-        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
-        {
-            return Collections.singletonList(value.bindAndGet(variables));
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return true;
-        }
-
-        public boolean isIN()
-        {
-            return false;
-        }
-
-        public boolean isOnToken()
-        {
-            return onToken;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("EQ(%s)%s", value, onToken ? "*" : "");
-        }
-    }
+    public static interface EQ extends Restriction {}
 
-    public static abstract class IN implements Restriction
+    public static interface IN extends Restriction
     {
-        public static IN create(List<Term> values)
-        {
-            return new WithValues(values);
-        }
-
-        public static IN create(Term value) throws InvalidRequestException
-        {
-            assert value instanceof Lists.Marker; // we shouldn't have got there otherwise
-            return new WithMarker((Lists.Marker)value);
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return true;
-        }
-
-        // Used when we need to know if it's a IN with just one value before we have
-        // the bind variables. This is ugly and only there for backward compatiblity
-        // because we used to treate IN with 1 value like an EQ and need to preserve
-        // this behavior.
-        public abstract boolean canHaveOnlyOneValue();
-
-        public boolean isOnToken()
-        {
-            return false;
-        }
-
-        private static class WithValues extends IN
-        {
-            private final List<Term> values;
-
-            private WithValues(List<Term> values)
-            {
-                this.values = values;
-            }
-
-            public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
-            {
-                List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(values.size());
-                for (Term value : values)
-                    buffers.add(value.bindAndGet(variables));
-                return buffers;
-            }
-
-            public boolean canHaveOnlyOneValue()
-            {
-                return values.size() == 1;
-            }
-
-            @Override
-            public String toString()
-            {
-                return String.format("IN(%s)", values);
-            }
-        }
-
-        private static class WithMarker extends IN
-        {
-            private final Lists.Marker marker;
-
-            private WithMarker(Lists.Marker marker)
-            {
-                this.marker = marker;
-            }
-
-            public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
-            {
-                Lists.Value lval = marker.bind(variables);
-                if (lval == null)
-                    throw new InvalidRequestException("Invalid null value for IN restriction");
-                return lval.elements;
-            }
-
-            public boolean canHaveOnlyOneValue()
-            {
-                return false;
-            }
-
-            @Override
-            public String toString()
-            {
-                return "IN ?";
-            }
-        }
+        public boolean canHaveOnlyOneValue();
     }
 
-    public static class Slice implements Restriction
+    public static interface Slice extends Restriction
     {
-        private final Term[] bounds;
-        private final boolean[] boundInclusive;
-        private final boolean onToken;
-
-        // The name of the column that was preceding this one if the tuple notation of #4851 was used
-        // (note: if it is set for both bound, we'll validate both have the same previous value, but we
-        // still need to distinguish if it's set or not for both bound)
-        private final ColumnIdentifier[] previous;
-
-        public Slice(boolean onToken)
-        {
-            this.bounds = new Term[2];
-            this.boundInclusive = new boolean[2];
-            this.previous = new ColumnIdentifier[2];
-            this.onToken = onToken;
-        }
-
-        public boolean isSlice()
-        {
-            return true;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return false;
-        }
-
-        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public boolean isOnToken()
-        {
-            return onToken;
-        }
-
-        public boolean hasBound(Bound b)
-        {
-            return bounds[b.idx] != null;
-        }
-
-        public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
-        {
-            return bounds[b.idx].bindAndGet(variables);
-        }
-
-        public boolean isInclusive(Bound b)
-        {
-            return bounds[b.idx] == null || boundInclusive[b.idx];
-        }
-
-        public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound)
-        {
-            switch (eocBound)
-            {
-                case START:
-                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT;
-                case END:
-                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT;
-            }
-            throw new AssertionError();
-        }
-
-        public IndexOperator getIndexOperator(Bound b)
-        {
-            switch (b)
-            {
-                case START:
-                    return boundInclusive[b.idx] ? IndexOperator.GTE : IndexOperator.GT;
-                case END:
-                    return boundInclusive[b.idx] ? IndexOperator.LTE : IndexOperator.LT;
-            }
-            throw new AssertionError();
-        }
-
-        public void setBound(ColumnIdentifier name, Relation.Type type, Term t, ColumnIdentifier previousName) throws InvalidRequestException
-        {
-            Bound b;
-            boolean inclusive;
-            switch (type)
-            {
-                case GT:
-                    b = Bound.START;
-                    inclusive = false;
-                    break;
-                case GTE:
-                    b = Bound.START;
-                    inclusive = true;
-                    break;
-                case LT:
-                    b = Bound.END;
-                    inclusive = false;
-                    break;
-                case LTE:
-                    b = Bound.END;
-                    inclusive = true;
-                    break;
-                default:
-                    throw new AssertionError();
-            }
+        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException;
 
-            if (bounds[b.idx] != null)
-                throw new InvalidRequestException(String.format("Invalid restrictions found on %s", name));
+        /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
+        public boolean hasBound(Bound b);
 
-            bounds[b.idx] = t;
-            boundInclusive[b.idx] = inclusive;
+        public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException;
 
-            // If a bound is part of a tuple notation (#4851), the other bound must either also be or must not be set at all,
-            // and this even if there is a 2ndary index (it's not supported by the 2ndary code). And it's easier to validate
-            // this here so we do.
-            Bound reverse = Bound.reverse(b);
-            if (hasBound(reverse) && !(Objects.equal(previousName, previous[reverse.idx])))
-                throw new InvalidRequestException(String.format("Clustering column %s cannot be restricted both inside a tuple notation and outside it", name));
+        /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
+        public boolean isInclusive(Bound b);
 
-            previous[b.idx] = previousName;
-        }
+        public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound);
 
-        public boolean isPartOfTuple()
-        {
-            return previous[Bound.START.idx] != null || previous[Bound.END.idx] != null;
-        }
+        public IndexOperator getIndexOperator(Bound b);
 
-        @Override
-        public String toString()
-        {
-            return String.format("SLICE(%s %s, %s %s)%s", boundInclusive[0] ? ">=" : ">",
-                                                          bounds[0],
-                                                          boundInclusive[1] ? "<=" : "<",
-                                                          bounds[1],
-                                                          onToken ? "*" : "");
-        }
+        public void setBound(Relation.Type type, Term t) throws InvalidRequestException;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
index 337e8dc..94df854 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
@@ -73,7 +73,7 @@ public abstract class SchemaAlteringStatement extends CFStatement implements CQL
         return new ResultMessage.SchemaChange(changeType(), keyspace(), tableName);
     }
 
-    public ResultMessage executeInternal(QueryState state)
+    public ResultMessage executeInternal(QueryState state, QueryOptions options)
     {
         // executeInternal is for local query only, thus altering schema is not supported
         throw new UnsupportedOperationException();


[2/7] Support multi-row selects within a partition using IN

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 6b4309f..e339ccb 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -30,7 +30,6 @@ import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.*;
@@ -71,8 +70,13 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
     private final Selection selection;
     private final Term limit;
 
+    /** Restrictions on partitioning columns */
     private final Restriction[] keyRestrictions;
+
+    /** Restrictions on clustering columns */
     private final Restriction[] columnRestrictions;
+
+    /** Restrictions on non-primary key columns (i.e. secondary index restrictions) */
     private final Map<CFDefinition.Name, Restriction> metadataRestrictions = new HashMap<CFDefinition.Name, Restriction>();
 
     // The name of all restricted names not covered by the key or index filter
@@ -288,9 +292,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return rows;
     }
 
-    public ResultMessage.Rows executeInternal(QueryState state) throws RequestExecutionException, RequestValidationException
+    public ResultMessage.Rows executeInternal(QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException
     {
-        List<ByteBuffer> variables = Collections.emptyList();
+        List<ByteBuffer> variables = options.getValues();
         int limit = getLimit(variables);
         int limitForQuery = updateLimitForQuery(limit);
         long now = System.currentTimeMillis();
@@ -331,7 +335,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         if (keys.isEmpty()) // in case of IN () for (the last column of) the partition key.
             return null;
 
-        List<ReadCommand> commands = new ArrayList<ReadCommand>(keys.size());
+        List<ReadCommand> commands = new ArrayList<>(keys.size());
 
         IDiskAtomFilter filter = makeFilter(variables, limit);
         if (filter == null)
@@ -627,7 +631,11 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
     {
         assert onToken;
 
-        Restriction keyRestriction = keyRestrictions[0];
+        Restriction restriction = keyRestrictions[0];
+
+        assert !restriction.isMultiColumn() : "Unexpectedly got a multi-column restriction on a partition key for a range query";
+        SingleColumnRestriction keyRestriction = (SingleColumnRestriction)restriction;
+
         ByteBuffer value;
         if (keyRestriction.isEQ())
         {
@@ -635,7 +643,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         }
         else
         {
-            Restriction.Slice slice = (Restriction.Slice)keyRestriction;
+            SingleColumnRestriction.Slice slice = (SingleColumnRestriction.Slice)keyRestriction;
             if (!slice.hasBound(b))
                 return p.getMinimumToken();
 
@@ -654,7 +662,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             if (r == null)
                 return true;
             else if (r.isSlice())
-                return ((Restriction.Slice)r).isInclusive(b);
+            {
+                assert !r.isMultiColumn() : "Unexpectedly got multi-column restriction on partition key";
+                return ((SingleColumnRestriction.Slice)r).isInclusive(b);
+            }
         }
         // All equality
         return true;
@@ -687,7 +698,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         Iterator<CFDefinition.Name> idIter = cfDef.clusteringColumns().iterator();
         for (Restriction r : columnRestrictions)
         {
-            ColumnIdentifier id = idIter.next().name;
+            CFDefinition.Name name = idIter.next();
             assert r != null && !r.isSlice();
 
             List<ByteBuffer> values = r.values(variables);
@@ -695,7 +706,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             {
                 ByteBuffer val = values.get(0);
                 if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", id));
+                    throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", name.name));
                 builder.add(val);
             }
             else
@@ -712,7 +723,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     ByteBuffer val = iter.next();
                     ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
                     if (val == null)
-                        throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", id));
+                        throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", name.name));
                     b.add(val);
                     if (cfDef.isCompact)
                         columns.add(b.build());
@@ -790,6 +801,22 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                                         ColumnNameBuilder builder,
                                         List<ByteBuffer> variables) throws InvalidRequestException
     {
+
+        // check the first restriction to see if we're dealing with a multi-column restriction
+        if (!names.isEmpty())
+        {
+            Restriction firstRestriction = restrictions[0];
+            if (firstRestriction != null && firstRestriction.isMultiColumn())
+            {
+                if (firstRestriction.isSlice())
+                    return buildMultiColumnSliceBound(bound, names, (MultiColumnRestriction.Slice) firstRestriction, isReversed, builder, variables);
+                else if (firstRestriction.isIN())
+                    return buildMultiColumnInBound(bound, names, (MultiColumnRestriction.IN) firstRestriction, isReversed, builder, variables);
+                else
+                    return buildMultiColumnEQBound(bound, (MultiColumnRestriction.EQ) firstRestriction, isReversed, builder, variables);
+            }
+        }
+
         // The end-of-component of composite doesn't depend on whether the
         // component type is reversed or not (i.e. the ReversedType is applied
         // to the component comparator but not to the end-of-component itself),
@@ -813,23 +840,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                                                  ? builder.buildAsEndOfRange()
                                                  : builder.build());
             }
-
             if (r.isSlice())
             {
-                builder.add(getSliceValue(name, r, b, variables));
+                builder.add(getSliceValue(r, b, variables));
                 Relation.Type relType = ((Restriction.Slice)r).getRelation(eocBound, b);
-
-                // We can have more non null restriction if the "scalar" notation was used for the bound (#4851).
-                // In that case, we need to add them all, and end the cell name with the correct end-of-component.
-                while (iter.hasNext())
-                {
-                    name = iter.next();
-                    r = restrictions[name.position];
-                    if (isNullRestriction(r, b))
-                        break;
-
-                    builder.add(getSliceValue(name, r, b, variables));
-                }
                 return Collections.singletonList(builder.buildForRelation(relType));
             }
             else
@@ -837,11 +851,11 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 List<ByteBuffer> values = r.values(variables);
                 if (values.size() != 1)
                 {
-                    // IN query, we only support it on the clustering column
+                    // IN query, we only support it on the clustering columns
                     assert name.position == names.size() - 1;
                     // The IN query might not have listed the values in comparator order, so we need to re-sort
                     // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
-                    TreeSet<ByteBuffer> s = new TreeSet<ByteBuffer>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
+                    TreeSet<ByteBuffer> s = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
                     for (ByteBuffer val : values)
                     {
                         if (val == null)
@@ -850,7 +864,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                         // See below for why this
                         s.add((b == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
                     }
-                    return new ArrayList<ByteBuffer>(s);
+                    return new ArrayList<>(s);
                 }
 
                 ByteBuffer val = values.get(0);
@@ -868,18 +882,92 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return Collections.singletonList((bound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
     }
 
+    private List<ByteBuffer> buildMultiColumnSliceBound(Bound bound,
+                                                        Collection<CFDefinition.Name> names,
+                                                        MultiColumnRestriction.Slice slice,
+                                                        boolean isReversed,
+                                                        ColumnNameBuilder builder,
+                                                        List<ByteBuffer> variables) throws InvalidRequestException
+    {
+        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
+
+        Iterator<CFDefinition.Name> iter = names.iterator();
+        CFDefinition.Name firstName = iter.next();
+        // A hack to preserve pre-6875 behavior for tuple-notation slices where the comparator mixes ASCENDING
+        // and DESCENDING orders.  This stores the bound for the first component; we will re-use it for all following
+        // components, even if they don't match the first component's reversal/non-reversal.  Note that this does *not*
+        // guarantee correct query results, it just preserves the previous behavior.
+        Bound firstComponentBound = isReversed == isReversedType(firstName) ? bound : Bound.reverse(bound);
+
+        if (!slice.hasBound(firstComponentBound))
+            return Collections.singletonList(builder.componentCount() > 0 && eocBound == Bound.END
+                    ? builder.buildAsEndOfRange()
+                    : builder.build());
+
+        List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, variables);
+        builder.add(vals.get(firstName.position));
+
+        while(iter.hasNext())
+        {
+            CFDefinition.Name name = iter.next();
+            if (name.position >= vals.size())
+                break;
+
+            builder.add(vals.get(name.position));
+        }
+        Relation.Type relType = slice.getRelation(eocBound, firstComponentBound);
+        return Collections.singletonList(builder.buildForRelation(relType));
+    }
+
+    private List<ByteBuffer> buildMultiColumnInBound(Bound bound,
+                                                     Collection<CFDefinition.Name> names,
+                                                     MultiColumnRestriction.IN restriction,
+                                                     boolean isReversed,
+                                                     ColumnNameBuilder builder,
+                                                     List<ByteBuffer> variables) throws InvalidRequestException
+    {
+        List<List<ByteBuffer>> splitInValues = restriction.splitValues(variables);
+
+        // The IN query might not have listed the values in comparator order, so we need to re-sort
+        // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
+        TreeSet<ByteBuffer> inValues = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
+        Iterator<CFDefinition.Name> iter = names.iterator();
+        for (List<ByteBuffer> components : splitInValues)
+        {
+            ColumnNameBuilder nameBuilder = builder.copy();
+            for (ByteBuffer component : components)
+                nameBuilder.add(component);
+
+            Bound b = isReversed == isReversedType(iter.next()) ? bound : Bound.reverse(bound);
+            inValues.add((bound == Bound.END && nameBuilder.remainingCount() > 0) ? nameBuilder.buildAsEndOfRange() : nameBuilder.build());
+        }
+        return new ArrayList<>(inValues);
+    }
+
+    private List<ByteBuffer> buildMultiColumnEQBound(Bound bound, MultiColumnRestriction.EQ restriction, boolean isReversed, ColumnNameBuilder builder, List<ByteBuffer> variables) throws InvalidRequestException
+    {
+        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
+        for (ByteBuffer component : restriction.values(variables))
+            builder.add(component);
+
+        ByteBuffer result = builder.componentCount() > 0 && eocBound == Bound.END
+                ? builder.buildAsEndOfRange()
+                : builder.build();
+        return Collections.singletonList(result);
+    }
+
     private static boolean isNullRestriction(Restriction r, Bound b)
     {
         return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
     }
 
-    private static ByteBuffer getSliceValue(CFDefinition.Name name, Restriction r, Bound b, List<ByteBuffer> variables) throws InvalidRequestException
+    private static ByteBuffer getSliceValue(Restriction r, Bound b, List<ByteBuffer> variables) throws InvalidRequestException
     {
         Restriction.Slice slice = (Restriction.Slice)r;
         assert slice.hasBound(b);
         ByteBuffer val = slice.bound(b, variables);
         if (val == null)
-            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
+            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
         return val;
     }
 
@@ -923,11 +1011,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     if (slice.hasBound(b))
                     {
                         ByteBuffer value = slice.bound(b, variables);
-                        if (value == null)
-                            throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
-                        if (value.remaining() > 0xFFFF)
-                            throw new InvalidRequestException("Index expression values may not be larger than 64K");
-
+                        validateIndexExpressionValue(value, name);
                         IndexOperator op = slice.getIndexOperator(b);
                         // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
                         // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
@@ -946,16 +1030,21 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
 
                 ByteBuffer value = values.get(0);
-                if (value == null)
-                    throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
-                if (value.remaining() > 0xFFFF)
-                    throw new InvalidRequestException("Index expression values may not be larger than 64K");
+                validateIndexExpressionValue(value, name);
                 expressions.add(new IndexExpression(name.name.key, IndexOperator.EQ, value));
             }
         }
         return expressions;
     }
 
+    private void validateIndexExpressionValue(ByteBuffer value, CFDefinition.Name name) throws InvalidRequestException
+    {
+        if (value == null)
+            throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
+        if (value.remaining() > 0xFFFF)
+            throw new InvalidRequestException("Index expression values may not be larger than 64K");
+    }
+
     private static IndexOperator reverse(IndexOperator op)
     {
         switch (op)
@@ -1266,7 +1355,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
             CFDefinition cfDef = cfm.getCfDef();
 
-            VariableSpecifications names = getBoundVariables();
+            VariableSpecifications boundNames = getBoundVariables();
 
             // Select clause
             if (parameters.isCount && !selectClause.isEmpty())
@@ -1279,14 +1368,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             if (parameters.isDistinct)
                 validateDistinctSelection(selection.getColumns(), cfDef.partitionKeys());
 
-            Term prepLimit = null;
-            if (limit != null)
-            {
-                prepLimit = limit.prepare(limitReceiver());
-                prepLimit.collectMarkerSpecification(names);
-            }
-
-            SelectStatement stmt = new SelectStatement(cfDef, names.size(), parameters, selection, prepLimit);
+            SelectStatement stmt = new SelectStatement(cfDef, boundNames.size(), parameters, selection, prepareLimit(boundNames));
 
             /*
              * WHERE clause. For a given entity, rules are:
@@ -1298,52 +1380,318 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
              */
             boolean hasQueriableIndex = false;
             boolean hasQueriableClusteringColumnIndex = false;
-            for (Relation rel : whereClause)
+            for (Relation relation : whereClause)
             {
-                CFDefinition.Name name = cfDef.get(rel.getEntity());
-                if (name == null)
+                if (relation.isMultiColumn())
+                {
+                    MultiColumnRelation rel = (MultiColumnRelation) relation;
+                    List<CFDefinition.Name> names = new ArrayList<>(rel.getEntities().size());
+                    for (ColumnIdentifier entity : rel.getEntities())
+                    {
+                        boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
+                        hasQueriableIndex |= queriable[0];
+                        hasQueriableClusteringColumnIndex |= queriable[1];
+                        names.add(cfDef.get(entity));
+                    }
+                    updateRestrictionsForRelation(stmt, names, rel, boundNames);
+                }
+                else
+                {
+                    SingleColumnRelation rel = (SingleColumnRelation) relation;
+                    boolean[] queriable = processRelationEntity(stmt, relation, rel.getEntity(), cfDef);
+                    hasQueriableIndex |= queriable[0];
+                    hasQueriableClusteringColumnIndex |= queriable[1];
+                    updateRestrictionsForRelation(stmt, cfDef.get(rel.getEntity()), rel, boundNames);
+                }
+            }
+
+             // At this point, the select statement if fully constructed, but we still have a few things to validate
+            processPartitionKeyRestrictions(stmt, cfDef, hasQueriableIndex);
+
+            // All (or none) of the partition key columns have been specified;
+            // hence there is no need to turn these restrictions into index expressions.
+            if (!stmt.usesSecondaryIndexing)
+                stmt.restrictedNames.removeAll(cfDef.partitionKeys());
+
+            if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
+                throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
+
+            processColumnRestrictions(stmt, cfDef, hasQueriableIndex);
+
+            // Covers indexes on the first clustering column (among others).
+            if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
+                stmt.usesSecondaryIndexing = true;
+
+            if (!stmt.usesSecondaryIndexing)
+                stmt.restrictedNames.removeAll(cfDef.clusteringColumns());
+
+            // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
+            // there is restrictions not covered by the PK.
+            if (!stmt.metadataRestrictions.isEmpty())
+            {
+                if (!hasQueriableIndex)
+                    throw new InvalidRequestException("No indexed columns present in by-columns clause with Equal operator");
+                stmt.usesSecondaryIndexing = true;
+            }
+
+            if (stmt.usesSecondaryIndexing)
+                validateSecondaryIndexSelections(stmt);
+
+            if (!stmt.parameters.orderings.isEmpty())
+                processOrderingClause(stmt, cfDef);
+
+            checkNeedsFiltering(stmt);
+
+            return new ParsedStatement.Prepared(stmt, boundNames);
+        }
+
+        /** Returns a pair of (hasQueriableIndex, hasQueriableClusteringColumnIndex) */
+        private boolean[] processRelationEntity(SelectStatement stmt, Relation relation, ColumnIdentifier entity, CFDefinition cfDef) throws InvalidRequestException
+        {
+            CFDefinition.Name name = cfDef.get(entity);
+            if (name == null)
+                handleUnrecognizedEntity(entity, relation);
+
+            stmt.restrictedNames.add(name);
+            if (cfDef.cfm.getColumnDefinition(name.name.key).isIndexed() && relation.operator() == Relation.Type.EQ)
+                return new boolean[]{true, name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS};
+            return new boolean[]{false, false};
+        }
+
+        /** Throws an InvalidRequestException for an unrecognized identifier in the WHERE clause */
+        private void handleUnrecognizedEntity(ColumnIdentifier entity, Relation relation) throws InvalidRequestException
+        {
+            if (containsAlias(entity))
+                throw new InvalidRequestException(String.format("Aliases aren't allowed in the where clause ('%s')", relation));
+            else
+                throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", entity, relation));
+        }
+
+        /** Returns a Term for the limit or null if no limit is set */
+        private Term prepareLimit(VariableSpecifications boundNames) throws InvalidRequestException
+        {
+            if (limit == null)
+                return null;
+
+            Term prepLimit = limit.prepare(limitReceiver());
+            prepLimit.collectMarkerSpecification(boundNames);
+            return prepLimit;
+        }
+
+        private void updateRestrictionsForRelation(SelectStatement stmt, List<CFDefinition.Name> names, MultiColumnRelation relation, VariableSpecifications boundNames) throws InvalidRequestException
+        {
+            List<CFDefinition.Name> restrictedColumns = new ArrayList<>();
+            Set<CFDefinition.Name> seen = new HashSet<>();
+
+            int previousPosition = -1;
+            for (CFDefinition.Name name : names)
+            {
+                // ensure multi-column restriction only applies to clustering columns
+                if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
+                    throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", name));
+
+                if (seen.contains(name))
+                    throw new InvalidRequestException(String.format("Column \"%s\" appeared twice in a relation: %s", name, relation));
+                seen.add(name);
+
+                // check that no clustering columns were skipped
+                if (name.position != previousPosition + 1)
                 {
-                    if (containsAlias(rel.getEntity()))
-                        throw new InvalidRequestException(String.format("Aliases aren't allowed in where clause ('%s')", rel));
+                    if (previousPosition == -1)
+                        throw new InvalidRequestException(String.format(
+                                "Clustering columns may not be skipped in multi-column relations. " +
+                                "They should appear in the PRIMARY KEY order. Got %s", relation));
                     else
-                        throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", rel.getEntity(), rel));
+                        throw new InvalidRequestException(String.format(
+                                "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
                 }
+                previousPosition++;
 
-                ColumnDefinition def = cfDef.cfm.getColumnDefinition(name.name.key);
-                stmt.restrictedNames.add(name);
-                if (def.isIndexed() && rel.operator() == Relation.Type.EQ)
+                Restriction existing = getExistingRestriction(stmt, name);
+                Relation.Type operator = relation.operator();
+                if (existing != null)
                 {
-                    hasQueriableIndex = true;
-                    if (name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS)
-                        hasQueriableClusteringColumnIndex = true;
+                    if (operator == Relation.Type.EQ || operator == Relation.Type.IN)
+                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation", name, relation.operator()));
+                    else if (!existing.isSlice())
+                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", name));
                 }
+                restrictedColumns.add(name);
+            }
+
+            boolean onToken = false;
 
-                switch (name.kind)
+            switch (relation.operator())
+            {
+                case EQ:
                 {
-                    case KEY_ALIAS:
-                        stmt.keyRestrictions[name.position] = updateRestriction(cfm, name, stmt.keyRestrictions[name.position], rel, names);
-                        break;
-                    case COLUMN_ALIAS:
-                        stmt.columnRestrictions[name.position] = updateRestriction(cfm, name, stmt.columnRestrictions[name.position], rel, names);
-                        break;
-                    case VALUE_ALIAS:
-                        throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", name.name));
-                    case COLUMN_METADATA:
-                    case STATIC:
-                        // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
-                        Restriction r = updateRestriction(cfm, name, stmt.metadataRestrictions.get(name), rel, names);
-                        if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
-                            // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that slide.
-                            throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", name));
-                        stmt.metadataRestrictions.put(name, r);
-                        break;
+                    Term t = relation.getValue().prepare(names);
+                    t.collectMarkerSpecification(boundNames);
+                    Restriction restriction = new MultiColumnRestriction.EQ(t, onToken);
+                    for (CFDefinition.Name name : restrictedColumns)
+                        stmt.columnRestrictions[name.position] = restriction;
+                    break;
+                }
+                case IN:
+                {
+                    Restriction restriction;
+                    List<? extends Term.MultiColumnRaw> inValues = relation.getInValues();
+                    if (inValues != null)
+                    {
+                        // we have something like "(a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) or
+                        // "(a, b, c) IN (?, ?, ?)
+                        List<Term> terms = new ArrayList<>(inValues.size());
+                        for (Term.MultiColumnRaw tuple : inValues)
+                        {
+                            Term t = tuple.prepare(names);
+                            t.collectMarkerSpecification(boundNames);
+                            terms.add(t);
+                        }
+                         restriction = new MultiColumnRestriction.InWithValues(terms);
+                    }
+                    else
+                    {
+                        Tuples.INRaw rawMarker = relation.getInMarker();
+                        AbstractMarker t = rawMarker.prepare(names);
+                        t.collectMarkerSpecification(boundNames);
+                        restriction = new MultiColumnRestriction.InWithMarker(t);
+                    }
+                    for (CFDefinition.Name name : restrictedColumns)
+                        stmt.columnRestrictions[name.position] = restriction;
+
+                    break;
+                }
+                case LT:
+                case LTE:
+                case GT:
+                case GTE:
+                {
+                    Term t = relation.getValue().prepare(names);
+                    t.collectMarkerSpecification(boundNames);
+                    for (CFDefinition.Name name : names)
+                    {
+                        Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, name);
+                        if (restriction == null)
+                            restriction = new MultiColumnRestriction.Slice(onToken);
+                        else if (!restriction.isMultiColumn())
+                            throw new InvalidRequestException(String.format("Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities", name, relation));
+                        restriction.setBound(relation.operator(), t);
+                        stmt.columnRestrictions[name.position] = restriction;
+                    }
                 }
             }
+        }
 
-            /*
-             * At this point, the select statement if fully constructed, but we still have a few things to validate
-             */
+        private Restriction getExistingRestriction(SelectStatement stmt, CFDefinition.Name name)
+        {
+            switch (name.kind)
+            {
+                case KEY_ALIAS:
+                    return stmt.keyRestrictions[name.position];
+                case COLUMN_ALIAS:
+                    return stmt.columnRestrictions[name.position];
+                case VALUE_ALIAS:
+                    return null;
+                default:
+                    return stmt.metadataRestrictions.get(name);
+            }
+        }
 
+        private void updateRestrictionsForRelation(SelectStatement stmt, CFDefinition.Name name, SingleColumnRelation relation, VariableSpecifications names) throws InvalidRequestException
+        {
+            switch (name.kind)
+            {
+                case KEY_ALIAS:
+                    stmt.keyRestrictions[name.position] = updateSingleColumnRestriction(name, stmt.keyRestrictions[name.position], relation, names);
+                    break;
+                case COLUMN_ALIAS:
+                    stmt.columnRestrictions[name.position] = updateSingleColumnRestriction(name, stmt.columnRestrictions[name.position], relation, names);
+                    break;
+                case VALUE_ALIAS:
+                    throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", name.name));
+                case COLUMN_METADATA:
+                case STATIC:
+                    // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
+                    Restriction r = updateSingleColumnRestriction(name, stmt.metadataRestrictions.get(name), relation, names);
+                    if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
+                        // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that slide.
+                        throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", name));
+                    stmt.metadataRestrictions.put(name, r);
+                    break;
+            }
+        }
+
+        Restriction updateSingleColumnRestriction(CFDefinition.Name name, Restriction existingRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
+        {
+            ColumnSpecification receiver = name;
+            if (newRel.onToken)
+            {
+                if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS)
+                    throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", name));
+
+                receiver = new ColumnSpecification(name.ksName,
+                                                   name.cfName,
+                                                   new ColumnIdentifier("partition key token", true),
+                                                   StorageService.getPartitioner().getTokenValidator());
+            }
+
+            switch (newRel.operator())
+            {
+                case EQ:
+                {
+                    if (existingRestriction != null)
+                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", name));
+                    Term t = newRel.getValue().prepare(receiver);
+                    t.collectMarkerSpecification(boundNames);
+                    existingRestriction = new SingleColumnRestriction.EQ(t, newRel.onToken);
+                }
+                break;
+                case IN:
+                    if (existingRestriction != null)
+                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", name));
+
+                    if (newRel.getInValues() == null)
+                    {
+                        // Means we have a "SELECT ... IN ?"
+                        assert newRel.getValue() != null;
+                        Term t = newRel.getValue().prepare(receiver);
+                        t.collectMarkerSpecification(boundNames);
+                        existingRestriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
+                    }
+                    else
+                    {
+                        List<Term> inValues = new ArrayList<Term>(newRel.getInValues().size());
+                        for (Term.Raw raw : newRel.getInValues())
+                        {
+                            Term t = raw.prepare(receiver);
+                            t.collectMarkerSpecification(boundNames);
+                            inValues.add(t);
+                        }
+                        existingRestriction = new SingleColumnRestriction.InWithValues(inValues);
+                    }
+                    break;
+                case GT:
+                case GTE:
+                case LT:
+                case LTE:
+                {
+                    if (existingRestriction == null)
+                        existingRestriction = new SingleColumnRestriction.Slice(newRel.onToken);
+                    else if (!existingRestriction.isSlice())
+                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both an equality and an inequality relation", name));
+                    else if (existingRestriction.isMultiColumn())
+                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both a tuple notation inequality and a single column inequality (%s)", name, newRel));
+                    Term t = newRel.getValue().prepare(receiver);
+                    t.collectMarkerSpecification(boundNames);
+                    ((SingleColumnRestriction.Slice)existingRestriction).setBound(newRel.operator(), t);
+                }
+                break;
+            }
+            return existingRestriction;
+        }
+
+        private void processPartitionKeyRestrictions(SelectStatement stmt, CFDefinition cfDef, boolean hasQueriableIndex) throws InvalidRequestException
+        {
             // If there is a queriable index, no special condition are required on the other restrictions.
             // But we still need to know 2 things:
             //   - If we don't have a queriable index, is the query ok
@@ -1386,7 +1734,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                         stmt.usesSecondaryIndexing = true;
                         break;
                     }
-                    throw new InvalidRequestException(String.format("partition key part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
+                    throw new InvalidRequestException(String.format(
+                            "Partitioning column \"%s\" cannot be restricted because the preceding column (\"%s\") is " +
+                            "either not restricted or is restricted by a non-EQ relation", cname, previous));
                 }
                 else if (restriction.isOnToken())
                 {
@@ -1418,22 +1768,17 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 }
                 previous = cname;
             }
+        }
 
-            // All (or none) of the partition key columns have been specified;
-            // hence there is no need to turn these restrictions into index expressions.
-            if (!stmt.usesSecondaryIndexing)
-                stmt.restrictedNames.removeAll(cfDef.partitionKeys());
-
-            if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
-                throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
-
+        private void processColumnRestrictions(SelectStatement stmt, CFDefinition cfDef, boolean hasQueriableIndex) throws InvalidRequestException
+        {
             // If a clustering key column is restricted by a non-EQ relation, all preceding
             // columns must have a EQ, and all following must have no restriction. Unless
             // the column is indexed that is.
-            canRestrictFurtherComponents = true;
-            previous = null;
+            boolean canRestrictFurtherComponents = true;
+            CFDefinition.Name previous = null;
             boolean previousIsSlice = false;
-            iter = cfDef.clusteringColumns().iterator();
+            Iterator<CFDefinition.Name> iter = cfDef.clusteringColumns().iterator();
             for (int i = 0; i < stmt.columnRestrictions.length; i++)
             {
                 CFDefinition.Name cname = iter.next();
@@ -1451,20 +1796,21 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     //   1) we're in the special case of the 'tuple' notation from #4851 which we expand as multiple
                     //      consecutive slices: in which case we're good with this restriction and we continue
                     //   2) we have a 2ndary index, in which case we have to use it but can skip more validation
-                    if (!(previousIsSlice && restriction.isSlice() && ((Restriction.Slice)restriction).isPartOfTuple()))
+                    if (!(previousIsSlice && restriction.isSlice() && restriction.isMultiColumn()))
                     {
                         if (hasQueriableIndex)
                         {
                             stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
                             break;
                         }
-                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
+                        throw new InvalidRequestException(String.format(
+                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is either not restricted or by a non-EQ relation)", cname, previous));
                     }
                 }
                 else if (restriction.isSlice())
                 {
-                    canRestrictFurtherComponents = false;
                     previousIsSlice = true;
+                    canRestrictFurtherComponents = false;
                     Restriction.Slice slice = (Restriction.Slice)restriction;
                     // For non-composite slices, we don't support internally the difference between exclusive and
                     // inclusive bounds, so we deal with it manually.
@@ -1473,156 +1819,148 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 }
                 else if (restriction.isIN())
                 {
-                    // We only support IN for the last name and for compact storage so far
-                    // TODO: #3885 allows us to extend to non compact as well, but that remains to be done
-                    if (i != stmt.columnRestrictions.length - 1)
-                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cname));
-                    else if (stmt.selectACollection())
-                        throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cname));
+                    if (!restriction.isMultiColumn() && i != stmt.columnRestrictions.length - 1)
+                        throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cname));
+                    if (stmt.selectACollection())
+                        throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cname));
                 }
 
                 previous = cname;
             }
+        }
 
-            // Covers indexes on the first clustering column (among others).
-            if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
-                stmt.usesSecondaryIndexing = true;
-
-            if (!stmt.usesSecondaryIndexing)
-                stmt.restrictedNames.removeAll(cfDef.clusteringColumns());
-
-            // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
-            // there is restrictions not covered by the PK.
-            if (!stmt.metadataRestrictions.isEmpty())
-            {
-                if (!hasQueriableIndex)
-                    throw new InvalidRequestException("No indexed columns present in by-columns clause with Equal operator");
-                stmt.usesSecondaryIndexing = true;
-            }
+        private void validateSecondaryIndexSelections(SelectStatement stmt) throws InvalidRequestException
+        {
+            if (stmt.keyIsInRelation)
+                throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
+            // When the user only select static columns, the intent is that we don't query the whole partition but just
+            // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
+            // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
+            if (stmt.selectsOnlyStaticColumns)
+                throw new InvalidRequestException("Queries using 2ndary indexes don't support selecting only static columns");
+        }
 
+        private void verifyOrderingIsAllowed(SelectStatement stmt) throws InvalidRequestException
+        {
             if (stmt.usesSecondaryIndexing)
-            {
-                if (stmt.keyIsInRelation)
-                    throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
-                // When the user only select static columns, the intent is that we don't query the whole partition but just
-                // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
-                // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
-                if (stmt.selectsOnlyStaticColumns)
-                    throw new InvalidRequestException("Queries using 2ndary indexes don't support selecting only static columns");
-            }
+                throw new InvalidRequestException("ORDER BY with 2ndary indexes is not supported.");
 
-            if (!stmt.parameters.orderings.isEmpty())
-            {
-                if (stmt.usesSecondaryIndexing)
-                    throw new InvalidRequestException("ORDER BY with 2ndary indexes is not supported.");
+            if (stmt.isKeyRange)
+                throw new InvalidRequestException("ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
+        }
 
-                if (stmt.isKeyRange)
-                    throw new InvalidRequestException("ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
+        private void handleUnrecognizedOrderingColumn(ColumnIdentifier column) throws InvalidRequestException
+        {
+            if (containsAlias(column))
+                throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
+            else
+                throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
+        }
 
-                // If we order an IN query, we'll have to do a manual sort post-query. Currently, this sorting requires that we
-                // have queried the column on which we sort (TODO: we should update it to add the column on which we sort to the one
-                // queried automatically, and then removing it from the resultSet afterwards if needed)
-                if (stmt.keyIsInRelation)
+        private void processOrderingClause(SelectStatement stmt, CFDefinition cfDef) throws InvalidRequestException
+        {
+            verifyOrderingIsAllowed(stmt);
+
+            // If we order an IN query, we'll have to do a manual sort post-query. Currently, this sorting requires that we
+            // have queried the column on which we sort (TODO: we should update it to add the column on which we sort to the one
+            // queried automatically, and then removing it from the resultSet afterwards if needed)
+            if (stmt.keyIsInRelation)
+            {
+                stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
+                for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
                 {
-                    stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
-                    for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
-                    {
-                        final CFDefinition.Name name = cfDef.get(column);
-                        if (name == null)
-                        {
-                            if (containsAlias(column))
-                                throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
-                            else
-                                throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
-                        }
+                    final CFDefinition.Name name = cfDef.get(column);
+                    if (name == null)
+                        handleUnrecognizedOrderingColumn(column);
 
-                        if (selectClause.isEmpty()) // wildcard
+                    if (selectClause.isEmpty()) // wildcard
+                    {
+                        stmt.orderingIndexes.put(name, Iterables.indexOf(cfDef, new Predicate<CFDefinition.Name>()
                         {
-                            stmt.orderingIndexes.put(name, Iterables.indexOf(cfDef, new Predicate<CFDefinition.Name>()
-                                                                                    {
-                                                                                        public boolean apply(CFDefinition.Name n)
-                                                                                        {
-                                                                                            return name.equals(n);
-                                                                                        }
-                                                                                    }));
-                        }
-                        else
+                            public boolean apply(CFDefinition.Name n)
+                            {
+                                return name.equals(n);
+                            }
+                        }));
+                    }
+                    else
+                    {
+                        boolean hasColumn = false;
+                        for (int i = 0; i < selectClause.size(); i++)
                         {
-                            boolean hasColumn = false;
-                            for (int i = 0; i < selectClause.size(); i++)
+                            RawSelector selector = selectClause.get(i);
+                            if (name.name.equals(selector.selectable))
                             {
-                                RawSelector selector = selectClause.get(i);
-                                if (name.name.equals(selector.selectable))
-                                {
-                                    stmt.orderingIndexes.put(name, i);
-                                    hasColumn = true;
-                                    break;
-                                }
+                                stmt.orderingIndexes.put(name, i);
+                                hasColumn = true;
+                                break;
                             }
-
-                            if (!hasColumn)
-                                throw new InvalidRequestException("ORDER BY could not be used on columns missing in select clause.");
                         }
+
+                        if (!hasColumn)
+                            throw new InvalidRequestException("ORDER BY could not be used on columns missing in select clause.");
                     }
                 }
+            }
+            stmt.isReversed = isReversed(stmt, cfDef);
+        }
 
-                Boolean[] reversedMap = new Boolean[cfDef.clusteringColumnsCount()];
-                int i = 0;
-                for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
-                {
-                    ColumnIdentifier column = entry.getKey();
-                    boolean reversed = entry.getValue();
+        private boolean isReversed(SelectStatement stmt, CFDefinition cfDef) throws InvalidRequestException
+        {
+            Boolean[] reversedMap = new Boolean[cfDef.clusteringColumnsCount()];
+            int i = 0;
+            for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
+            {
+                ColumnIdentifier column = entry.getKey();
+                boolean reversed = entry.getValue();
 
-                    CFDefinition.Name name = cfDef.get(column);
-                    if (name == null)
-                    {
-                        if (containsAlias(column))
-                            throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
-                        else
-                            throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
-                    }
+                CFDefinition.Name name = cfDef.get(column);
+                if (name == null)
+                    handleUnrecognizedOrderingColumn(column);
 
-                    if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
-                        throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
+                if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
+                    throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
 
-                    if (i++ != name.position)
-                        throw new InvalidRequestException(String.format("Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"));
+                if (i++ != name.position)
+                    throw new InvalidRequestException(String.format("Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"));
 
-                    reversedMap[name.position] = (reversed != isReversedType(name));
-                }
+                reversedMap[name.position] = (reversed != isReversedType(name));
+            }
 
-                // Check that all boolean in reversedMap, if set, agrees
-                Boolean isReversed = null;
-                for (Boolean b : reversedMap)
-                {
-                    // Column on which order is specified can be in any order
-                    if (b == null)
-                        continue;
+            // Check that all boolean in reversedMap, if set, agrees
+            Boolean isReversed = null;
+            for (Boolean b : reversedMap)
+            {
+                // Column on which order is specified can be in any order
+                if (b == null)
+                    continue;
 
-                    if (isReversed == null)
-                    {
-                        isReversed = b;
-                        continue;
-                    }
-                    if (isReversed != b)
-                        throw new InvalidRequestException(String.format("Unsupported order by relation"));
+                if (isReversed == null)
+                {
+                    isReversed = b;
+                    continue;
                 }
-                assert isReversed != null;
-                stmt.isReversed = isReversed;
+                if (isReversed != b)
+                    throw new InvalidRequestException(String.format("Unsupported order by relation"));
             }
+            assert isReversed != null;
+            return isReversed;
+        }
 
-            // Make sure this queries is allowed (note: non key range non indexed cannot involve filtering underneath)
+        /** If ALLOW FILTERING was not specified, this verifies that it is not needed */
+        private void checkNeedsFiltering(SelectStatement stmt) throws InvalidRequestException
+        {
+            // non-key-range non-indexed queries cannot involve filtering underneath
             if (!parameters.allowFiltering && (stmt.isKeyRange || stmt.usesSecondaryIndexing))
             {
                 // We will potentially filter data if either:
                 //  - Have more than one IndexExpression
                 //  - Have no index expression and the column filter is not the identity
                 if (stmt.restrictedNames.size() > 1 || (stmt.restrictedNames.isEmpty() && !stmt.columnFilterIsIdentity()))
-                    throw new InvalidRequestException("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. "
-                                                    + "If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING");
+                    throw new InvalidRequestException("Cannot execute this query as it might involve data filtering and " +
+                                                      "thus may have unpredictable performance. If you want to execute " +
+                                                      "this query despite the performance unpredictability, use ALLOW FILTERING");
             }
-
-            return new ParsedStatement.Prepared(stmt, names);
         }
 
         private void validateDistinctSelection(Collection<CFDefinition.Name> requestedColumns, Collection<CFDefinition.Name> partitionKey)
@@ -1653,79 +1991,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
         }
 
-        Restriction updateRestriction(CFMetaData cfm, CFDefinition.Name name, Restriction restriction, Relation newRel, VariableSpecifications boundNames) throws InvalidRequestException
-        {
-            ColumnSpecification receiver = name;
-            if (newRel.onToken)
-            {
-                if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS)
-                    throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", name));
-
-                receiver = new ColumnSpecification(name.ksName,
-                                                   name.cfName,
-                                                   new ColumnIdentifier("partition key token", true),
-                                                   StorageService.getPartitioner().getTokenValidator());
-            }
-
-            // We can only use the tuple notation of #4851 on clustering columns for now
-            if (newRel.previousInTuple != null && name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
-                throw new InvalidRequestException(String.format("Tuple notation can only be used on clustering columns but found on %s", name));
-
-            switch (newRel.operator())
-            {
-                case EQ:
-                    {
-                        if (restriction != null)
-                            throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", name));
-                        Term t = newRel.getValue().prepare(receiver);
-                        t.collectMarkerSpecification(boundNames);
-                        restriction = new Restriction.EQ(t, newRel.onToken);
-                    }
-                    break;
-                case IN:
-                    if (restriction != null)
-                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", name));
-
-                    if (newRel.getInValues() == null)
-                    {
-                        // Means we have a "SELECT ... IN ?"
-                        assert newRel.getValue() != null;
-                        Term t = newRel.getValue().prepare(receiver);
-                        t.collectMarkerSpecification(boundNames);
-                        restriction = Restriction.IN.create(t);
-                    }
-                    else
-                    {
-                        List<Term> inValues = new ArrayList<Term>(newRel.getInValues().size());
-                        for (Term.Raw raw : newRel.getInValues())
-                        {
-                            Term t = raw.prepare(receiver);
-                            t.collectMarkerSpecification(boundNames);
-                            inValues.add(t);
-                        }
-                        restriction = Restriction.IN.create(inValues);
-                    }
-                    break;
-                case GT:
-                case GTE:
-                case LT:
-                case LTE:
-                    {
-                        if (restriction == null)
-                            restriction = new Restriction.Slice(newRel.onToken);
-                        else if (!restriction.isSlice())
-                            throw new InvalidRequestException(String.format("%s cannot be restricted by both an equal and an inequal relation", name));
-                        Term t = newRel.getValue().prepare(receiver);
-                        t.collectMarkerSpecification(boundNames);
-                        if (newRel.previousInTuple != null && (name.position == 0 || !cfm.clusteringKeyColumns().get(name.position - 1).name.equals(newRel.previousInTuple.key)))
-                            throw new InvalidRequestException(String.format("Invalid tuple notation, column %s is not before column %s in the clustering order", newRel.previousInTuple, name.name));
-                        ((Restriction.Slice)restriction).setBound(name.name, newRel.operator(), t, newRel.previousInTuple);
-                    }
-                    break;
-            }
-            return restriction;
-        }
-
         @Override
         public String toString()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
new file mode 100644
index 0000000..2e63272
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
@@ -0,0 +1,300 @@
+/*
+ * 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.statements;
+
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.thrift.IndexOperator;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public abstract class SingleColumnRestriction implements Restriction
+{
+    public boolean isMultiColumn()
+    {
+        return false;
+    }
+
+    public static class EQ extends SingleColumnRestriction implements Restriction.EQ
+    {
+        protected final Term value;
+        private final boolean onToken;
+
+        public EQ(Term value, boolean onToken)
+        {
+            this.value = value;
+            this.onToken = onToken;
+        }
+
+        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            return Collections.singletonList(value.bindAndGet(variables));
+        }
+
+        public boolean isSlice()
+        {
+            return false;
+        }
+
+        public boolean isEQ()
+        {
+            return true;
+        }
+
+        public boolean isIN()
+        {
+            return false;
+        }
+
+        public boolean isOnToken()
+        {
+            return onToken;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("EQ(%s)%s", value, onToken ? "*" : "");
+        }
+    }
+
+    public static class InWithValues extends SingleColumnRestriction implements Restriction.IN
+    {
+        protected final List<Term> values;
+
+        public InWithValues(List<Term> values)
+        {
+            this.values = values;
+        }
+
+        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            List<ByteBuffer> buffers = new ArrayList<>(values.size());
+            for (Term value : values)
+                buffers.add(value.bindAndGet(variables));
+            return buffers;
+        }
+
+        public boolean canHaveOnlyOneValue()
+        {
+            return values.size() == 1;
+        }
+
+        public boolean isSlice()
+        {
+            return false;
+        }
+
+        public boolean isEQ()
+        {
+            return false;
+        }
+
+        public boolean isIN()
+        {
+            return true;
+        }
+
+        public boolean isOnToken()
+        {
+            return false;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("IN(%s)", values);
+        }
+    }
+
+    public static class InWithMarker extends SingleColumnRestriction implements Restriction.IN
+    {
+        protected final AbstractMarker marker;
+
+        public InWithMarker(AbstractMarker marker)
+        {
+            this.marker = marker;
+        }
+
+        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            Term.MultiItemTerminal lval = (Term.MultiItemTerminal)marker.bind(variables);
+            if (lval == null)
+                throw new InvalidRequestException("Invalid null value for IN restriction");
+            return lval.getElements();
+        }
+
+        public boolean canHaveOnlyOneValue()
+        {
+            return false;
+        }
+
+        public boolean isSlice()
+        {
+            return false;
+        }
+
+        public boolean isEQ()
+        {
+            return false;
+        }
+
+        public boolean isIN()
+        {
+            return true;
+        }
+
+        public boolean isOnToken()
+        {
+            return false;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "IN ?";
+        }
+    }
+
+    public static class Slice extends SingleColumnRestriction implements Restriction.Slice
+    {
+        protected final Term[] bounds;
+        protected final boolean[] boundInclusive;
+        protected final boolean onToken;
+
+        public Slice(boolean onToken)
+        {
+            this.bounds = new Term[2];
+            this.boundInclusive = new boolean[2];
+            this.onToken = onToken;
+        }
+
+        public boolean isSlice()
+        {
+            return true;
+        }
+
+        public boolean isEQ()
+        {
+            return false;
+        }
+
+        public boolean isIN()
+        {
+            return false;
+        }
+
+        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public boolean isOnToken()
+        {
+            return onToken;
+        }
+
+        /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
+        public boolean hasBound(Bound b)
+        {
+            return bounds[b.idx] != null;
+        }
+
+        public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            return bounds[b.idx].bindAndGet(variables);
+        }
+
+        /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
+        public boolean isInclusive(Bound b)
+        {
+            return bounds[b.idx] == null || boundInclusive[b.idx];
+        }
+
+        public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound)
+        {
+            switch (eocBound)
+            {
+                case START:
+                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT;
+                case END:
+                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT;
+            }
+            throw new AssertionError();
+        }
+
+        public IndexOperator getIndexOperator(Bound b)
+        {
+            switch (b)
+            {
+                case START:
+                    return boundInclusive[b.idx] ? IndexOperator.GTE : IndexOperator.GT;
+                case END:
+                    return boundInclusive[b.idx] ? IndexOperator.LTE : IndexOperator.LT;
+            }
+            throw new AssertionError();
+        }
+
+        public void setBound(Relation.Type type, Term t) throws InvalidRequestException
+        {
+            Bound b;
+            boolean inclusive;
+            switch (type)
+            {
+                case GT:
+                    b = Bound.START;
+                    inclusive = false;
+                    break;
+                case GTE:
+                    b = Bound.START;
+                    inclusive = true;
+                    break;
+                case LT:
+                    b = Bound.END;
+                    inclusive = false;
+                    break;
+                case LTE:
+                    b = Bound.END;
+                    inclusive = true;
+                    break;
+                default:
+                    throw new AssertionError();
+            }
+
+            if (bounds[b.idx] != null)
+                throw new InvalidRequestException(String.format(
+                        "More than one restriction was found for the %s bound", b.name().toLowerCase()));
+
+            bounds[b.idx] = t;
+            boundInclusive[b.idx] = inclusive;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("SLICE(%s %s, %s %s)%s", boundInclusive[0] ? ">=" : ">",
+                                 bounds[0],
+                                 boundInclusive[1] ? "<=" : "<",
+                                 bounds[1],
+                                 onToken ? "*" : "");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 30e57d5..ef1c4a4 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -77,7 +77,7 @@ public class TruncateStatement extends CFStatement implements CQLStatement
         return null;
     }
 
-    public ResultMessage executeInternal(QueryState state)
+    public ResultMessage executeInternal(QueryState state, QueryOptions options)
     {
         throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
index ee70f9d..efda72d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
@@ -59,7 +59,7 @@ public class UseStatement extends ParsedStatement implements CQLStatement
         return new ResultMessage.SetKeyspace(keyspace);
     }
 
-    public ResultMessage executeInternal(QueryState state)
+    public ResultMessage executeInternal(QueryState state, QueryOptions options)
     {
         // Internal queries are exclusively on the system keyspace and 'use' is thus useless
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43496384/src/java/org/apache/cassandra/db/marshal/TupleType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java
new file mode 100644
index 0000000..74211c8
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java
@@ -0,0 +1,279 @@
+/*
+ * 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.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import com.google.common.base.Objects;
+
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.serializers.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * This is essentially like a CompositeType, but it's not primarily meant for comparison, just
+ * to pack multiple values together so has a more friendly encoding.
+ */
+public class TupleType extends AbstractType<ByteBuffer>
+{
+    protected final List<AbstractType<?>> types;
+
+    public TupleType(List<AbstractType<?>> types)
+    {
+        this.types = types;
+    }
+
+    public static TupleType getInstance(TypeParser parser) throws ConfigurationException, SyntaxException
+    {
+        return new TupleType(parser.getTypeParameters());
+    }
+
+    public AbstractType<?> type(int i)
+    {
+        return types.get(i);
+    }
+
+    public int size()
+    {
+        return types.size();
+    }
+
+    public List<AbstractType<?>> allTypes()
+    {
+        return types;
+    }
+
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        if (!o1.hasRemaining() || !o2.hasRemaining())
+            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+
+        ByteBuffer bb1 = o1.duplicate();
+        ByteBuffer bb2 = o2.duplicate();
+
+        int i = 0;
+        while (bb1.remaining() > 0 && bb2.remaining() > 0)
+        {
+            AbstractType<?> comparator = types.get(i);
+
+            int size1 = bb1.getInt();
+            int size2 = bb2.getInt();
+
+            // Handle nulls
+            if (size1 < 0)
+            {
+                if (size2 < 0)
+                    continue;
+                return -1;
+            }
+            if (size2 < 0)
+                return 1;
+
+            ByteBuffer value1 = ByteBufferUtil.readBytes(bb1, size1);
+            ByteBuffer value2 = ByteBufferUtil.readBytes(bb2, size2);
+            int cmp = comparator.compare(value1, value2);
+            if (cmp != 0)
+                return cmp;
+
+            ++i;
+        }
+
+        if (bb1.remaining() == 0)
+            return bb2.remaining() == 0 ? 0 : -1;
+
+        // bb1.remaining() > 0 && bb2.remaining() == 0
+        return 1;
+    }
+
+    @Override
+    public void validate(ByteBuffer bytes) throws MarshalException
+    {
+        ByteBuffer input = bytes.duplicate();
+        for (int i = 0; i < size(); i++)
+        {
+            // we allow the input to have less fields than declared so as to support field addition.
+            if (!input.hasRemaining())
+                return;
+
+            if (input.remaining() < 4)
+                throw new MarshalException(String.format("Not enough bytes to read size of %dth component", i));
+
+            int size = input.getInt();
+            // We don't handle null just yet, but we should fix that soon (CASSANDRA-7206)
+            if (size < 0)
+                throw new MarshalException("Nulls are not yet supported inside tuple values");
+
+            if (input.remaining() < size)
+                throw new MarshalException(String.format("Not enough bytes to read %dth component", i));
+
+            ByteBuffer field = ByteBufferUtil.readBytes(input, size);
+            types.get(i).validate(field);
+        }
+
+        // We're allowed to get less fields than declared, but not more
+        if (input.hasRemaining())
+            throw new MarshalException("Invalid remaining data after end of tuple value");
+    }
+
+    /**
+     * Split a tuple value into its component values.
+     */
+    public ByteBuffer[] split(ByteBuffer value)
+    {
+        ByteBuffer[] components = new ByteBuffer[size()];
+        ByteBuffer input = value.duplicate();
+        for (int i = 0; i < size(); i++)
+        {
+            if (!input.hasRemaining())
+                return Arrays.copyOfRange(components, 0, i);
+
+            int size = input.getInt();
+            components[i] = size < 0 ? null : ByteBufferUtil.readBytes(input, size);
+        }
+        return components;
+    }
+
+    public static ByteBuffer buildValue(ByteBuffer[] components)
+    {
+        int totalLength = 0;
+        for (ByteBuffer component : components)
+            totalLength += 4 + component.remaining();
+
+        ByteBuffer result = ByteBuffer.allocate(totalLength);
+        for (ByteBuffer component : components)
+        {
+            result.putInt(component.remaining());
+            result.put(component.duplicate());
+        }
+        result.rewind();
+        return result;
+    }
+
+    @Override
+    public String getString(ByteBuffer value)
+    {
+        StringBuilder sb = new StringBuilder();
+        ByteBuffer input = value.duplicate();
+        for (int i = 0; i < size(); i++)
+        {
+            if (!input.hasRemaining())
+                return sb.toString();
+
+            if (i > 0)
+                sb.append(":");
+
+            int size = input.getInt();
+            assert size >= 0; // We don't support nulls yet, but we will likely do with #7206 and we'll need
+                              // a way to represent it as a string (without it conflicting with a user value)
+            ByteBuffer field = ByteBufferUtil.readBytes(input, size);
+            // We use ':' as delimiter so escape it if it's in the generated string
+            sb.append(field == null ? "null" : type(i).getString(value).replaceAll(":", "\\\\:"));
+        }
+        return sb.toString();
+    }
+
+    public ByteBuffer fromString(String source)
+    {
+        // Split the input on non-escaped ':' characters
+        List<String> strings = AbstractCompositeType.split(source);
+        ByteBuffer[] components = new ByteBuffer[strings.size()];
+        for (int i = 0; i < strings.size(); i++)
+        {
+            // TODO: we'll need to handle null somehow here once we support them
+            String str = strings.get(i).replaceAll("\\\\:", ":");
+            components[i] = type(i).fromString(str);
+        }
+        return buildValue(components);
+    }
+
+    public TypeSerializer<ByteBuffer> getSerializer()
+    {
+        return BytesSerializer.instance;
+    }
+
+    @Override
+    public boolean isCompatibleWith(AbstractType<?> previous)
+    {
+        if (!(previous instanceof TupleType))
+            return false;
+
+        // Extending with new components is fine, removing is not
+        TupleType tt = (TupleType)previous;
+        if (size() < tt.size())
+            return false;
+
+        for (int i = 0; i < tt.size(); i++)
+        {
+            AbstractType<?> tprev = tt.type(i);
+            AbstractType<?> tnew = type(i);
+            if (!tnew.isCompatibleWith(tprev))
+                return false;
+        }
+        return true;
+    }
+
+    @Override
+    public boolean isValueCompatibleWith(AbstractType<?> previous)
+    {
+        if (!(previous instanceof TupleType))
+            return false;
+
+        // Extending with new components is fine, removing is not
+        TupleType tt = (TupleType)previous;
+        if (size() < tt.size())
+            return false;
+
+        for (int i = 0; i < tt.size(); i++)
+        {
+            AbstractType<?> tprev = tt.type(i);
+            AbstractType<?> tnew = type(i);
+            if (!tnew.isValueCompatibleWith(tprev))
+                return false;
+        }
+        return true;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(types);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if(!(o instanceof TupleType))
+            return false;
+
+        TupleType that = (TupleType)o;
+        return types.equals(that.types);
+    }
+
+    @Override
+    public String toString()
+    {
+        return getClass().getName() + TypeParser.stringifyTypeParameters(types);
+    }
+}
+


[6/7] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by ty...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	CHANGES.txt
	src/java/org/apache/cassandra/cql3/Cql.g
	src/java/org/apache/cassandra/cql3/Lists.java
	src/java/org/apache/cassandra/cql3/QueryProcessor.java
	src/java/org/apache/cassandra/cql3/Relation.java
	src/java/org/apache/cassandra/cql3/ResultSet.java
	src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
	src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
	src/java/org/apache/cassandra/cql3/statements/Restriction.java
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java


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

Branch: refs/heads/trunk
Commit: bf5219000be9c03daa1dc4fb420b031f6ffec01d
Parents: 9bd3887 4349638
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Thu May 22 14:19:42 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Thu May 22 14:19:42 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 .../apache/cassandra/cql3/AbstractMarker.java   |   10 +-
 .../org/apache/cassandra/cql3/Constants.java    |    6 +
 src/java/org/apache/cassandra/cql3/Cql.g        |  109 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |   13 +-
 .../cassandra/cql3/MultiColumnRelation.java     |  144 +++
 .../org/apache/cassandra/cql3/Relation.java     |  104 +-
 .../cassandra/cql3/SingleColumnRelation.java    |   95 ++
 src/java/org/apache/cassandra/cql3/Term.java    |   10 +
 src/java/org/apache/cassandra/cql3/Tuples.java  |  349 ++++++
 .../cql3/statements/ModificationStatement.java  |   20 +-
 .../cql3/statements/MultiColumnRestriction.java |  137 +++
 .../cassandra/cql3/statements/Restriction.java  |  395 +------
 .../cql3/statements/SelectStatement.java        |  805 ++++++++-----
 .../statements/SingleColumnRestriction.java     |  413 +++++++
 .../cassandra/db/composites/CBuilder.java       |    2 +
 .../cassandra/db/composites/Composites.java     |    2 +
 .../cassandra/db/composites/CompoundCType.java  |   13 +
 .../cassandra/db/composites/SimpleCType.java    |   10 +
 .../apache/cassandra/db/marshal/TupleType.java  |  279 +++++
 .../cassandra/cql3/MultiColumnRelationTest.java | 1114 ++++++++++++++++++
 21 files changed, 3263 insertions(+), 768 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 55fc400,c6c51c3..7ab411b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -64,7 -35,24 +64,8 @@@ Merged from 2.0
   * Fix 2ndary index queries with DESC clustering order (CASSANDRA-6950)
   * Invalid key cache entries on DROP (CASSANDRA-6525)
   * Fix flapping RecoveryManagerTest (CASSANDRA-7084)
 + * Add missing iso8601 patterns for date strings (6973)
++ * Support selecting multiple rows in a partition using IN (CASSANDRA-6875)
  Merged from 1.2:
   * Add Cloudstack snitch (CASSANDRA-7147)
   * Update system.peers correctly when relocating tokens (CASSANDRA-7126)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/AbstractMarker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/AbstractMarker.java
index 2b9c6c9,4329ed9..0b59ed4
--- a/src/java/org/apache/cassandra/cql3/AbstractMarker.java
+++ b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
@@@ -99,10 -103,10 +103,10 @@@ public abstract class AbstractMarker ex
          }
  
          @Override
 -        public AbstractMarker prepare(ColumnSpecification receiver) throws InvalidRequestException
 +        public AbstractMarker prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
          {
              if (receiver.type instanceof CollectionType)
-                 throw new InvalidRequestException("Invalid IN relation on collection column");
+                 throw new InvalidRequestException("Collection columns do not support IN relations");
  
              return new Lists.Marker(bindIndex, makeInReceiver(receiver));
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Cql.g
index 4c1f2dc,ceb2bde..57b61a5
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@@ -962,44 -894,82 +976,85 @@@ relationType returns [Relation.Type op
      ;
  
  relation[List<Relation> clauses]
-     : name=cident type=relationType t=term { $clauses.add(new Relation(name, type, t)); }
-     | K_TOKEN 
-         { List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>(); }
-           '(' name1=cident { l.add(name1); } ( ',' namen=cident { l.add(namen); })* ')'
-         type=relationType t=term
+     : name=cident type=relationType t=term { $clauses.add(new SingleColumnRelation(name, type, t)); }
+     | K_TOKEN l=tupleOfIdentifiers type=relationType t=term
          {
              for (ColumnIdentifier id : l)
-                 $clauses.add(new Relation(id, type, t, true));
+                 $clauses.add(new SingleColumnRelation(id, type, t, true));
          }
-     | name=cident K_IN { Term.Raw marker = null; } (QMARK { marker = newINBindVariables(null); } | ':' mid=cident { marker = newINBindVariables(mid); })
-         { $clauses.add(new Relation(name, Relation.Type.IN, marker)); }
-     | name=cident K_IN { Relation rel = Relation.createInRelation($name.id); }
-        '(' ( f1=term { rel.addInValue(f1); } (',' fN=term { rel.addInValue(fN); } )* )? ')' { $clauses.add(rel); }
+     | name=cident K_IN marker=inMarker
+         { $clauses.add(new SingleColumnRelation(name, Relation.Type.IN, marker)); }
+     | name=cident K_IN inValues=singleColumnInValues
+         { $clauses.add(SingleColumnRelation.createInRelation($name.id, inValues)); }
 +    | name=cident K_CONTAINS { Relation.Type rt = Relation.Type.CONTAINS; } (K_KEY { rt = Relation.Type.CONTAINS_KEY; })?
-         t=term { $clauses.add(new Relation(name, rt, t)); }
-     | {
-          List<ColumnIdentifier> ids = new ArrayList<ColumnIdentifier>();
-          List<Term.Raw> terms = new ArrayList<Term.Raw>();
-       }
-         '(' n1=cident { ids.add(n1); } (',' ni=cident { ids.add(ni); })* ')'
-         type=relationType
-         '(' t1=term { terms.add(t1); } (',' ti=term { terms.add(ti); })* ')'
-       {
-           if (type == Relation.Type.IN)
-               addRecognitionError("Cannot use IN relation with tuple notation");
-           if (ids.size() != terms.size())
-               addRecognitionError(String.format("Number of values (" + terms.size() + ") in tuple notation doesn't match the number of column names (" + ids.size() + ")"));
-           else
-               for (int i = 0; i < ids.size(); i++)
-                   $clauses.add(new Relation(ids.get(i), type, terms.get(i), i == 0 ? null : ids.get(i-1)));
-       }
++        t=term { $clauses.add(new SingleColumnRelation(name, rt, t)); }
+     | ids=tupleOfIdentifiers
+       ( K_IN
+           ( '(' ')'
+               { $clauses.add(MultiColumnRelation.createInRelation(ids, new ArrayList<Tuples.Literal>())); }
+           | tupleInMarker=inMarkerForTuple /* (a, b, c) IN ? */
+               { $clauses.add(MultiColumnRelation.createSingleMarkerInRelation(ids, tupleInMarker)); }
+           | literals=tupleOfTupleLiterals /* (a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) */
+               {
+                   $clauses.add(MultiColumnRelation.createInRelation(ids, literals));
+               }
+           | markers=tupleOfMarkersForTuples /* (a, b, c) IN (?, ?, ...) */
+               { $clauses.add(MultiColumnRelation.createInRelation(ids, markers)); }
+           )
+       | type=relationType literal=tupleLiteral /* (a, b, c) > (1, 2, 3) or (a, b, c) > (?, ?, ?) */
+           {
+               $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, literal));
+           }
+       | type=relationType tupleMarker=markerForTuple /* (a, b, c) >= ? */
+           { $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, tupleMarker)); }
+       )
      | '(' relation[$clauses] ')'
      ;
  
+ inMarker returns [AbstractMarker.INRaw marker]
+     : QMARK { $marker = newINBindVariables(null); }
+     | ':' name=cident { $marker = newINBindVariables(name); }
+     ;
+ 
+ tupleOfIdentifiers returns [List<ColumnIdentifier> ids]
+     @init { $ids = new ArrayList<ColumnIdentifier>(); }
+     : '(' n1=cident { $ids.add(n1); } (',' ni=cident { $ids.add(ni); })* ')'
+     ;
+ 
+ singleColumnInValues returns [List<Term.Raw> terms]
+     @init { $terms = new ArrayList<Term.Raw>(); }
+     : '(' ( t1 = term { $terms.add(t1); } (',' ti=term { $terms.add(ti); })* )? ')'
+     ;
+ 
+ tupleLiteral returns [Tuples.Literal literal]
+     @init { List<Term.Raw> terms = new ArrayList<>(); }
+     : '(' t1=term { terms.add(t1); } (',' ti=term { terms.add(ti); })* ')' { $literal = new Tuples.Literal(terms); }
+     ;
+ 
+ tupleOfTupleLiterals returns [List<Tuples.Literal> literals]
+     @init { $literals = new ArrayList<>(); }
+     : '(' t1=tupleLiteral { $literals.add(t1); } (',' ti=tupleLiteral { $literals.add(ti); })* ')'
+     ;
+ 
+ markerForTuple returns [Tuples.Raw marker]
+     : QMARK { $marker = newTupleBindVariables(null); }
+     | ':' name=cident { $marker = newTupleBindVariables(name); }
+     ;
+ 
+ tupleOfMarkersForTuples returns [List<Tuples.Raw> markers]
+     @init { $markers = new ArrayList<Tuples.Raw>(); }
+     : '(' m1=markerForTuple { $markers.add(m1); } (',' mi=markerForTuple { $markers.add(mi); })* ')'
+     ;
+ 
+ inMarkerForTuple returns [Tuples.INRaw marker]
+     : QMARK { $marker = newTupleINBindVariables(null); }
+     | ':' name=cident { $marker = newTupleINBindVariables(name); }
+     ;
+ 
 -comparatorType returns [CQL3Type t]
 -    : c=native_type     { $t = c; }
 +comparatorType returns [CQL3Type.Raw t]
 +    : n=native_type     { $t = CQL3Type.Raw.from(n); }
      | c=collection_type { $t = c; }
 +    | id=userTypeName  { $t = CQL3Type.Raw.userType(id); }
      | s=STRING_LITERAL
        {
          try {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Lists.java
index f12af88,d483dd5..c214fa8
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@@ -147,25 -144,18 +147,30 @@@ public abstract class List
              }
          }
  
 -        public ByteBuffer get()
 +        public ByteBuffer get(QueryOptions options)
          {
 -            return CollectionType.pack(elements, elements.size());
 +            return CollectionSerializer.pack(elements, elements.size(), options.getProtocolVersion());
 +        }
 +
 +        public boolean equals(ListType lt, Value v)
 +        {
 +            if (elements.size() != v.elements.size())
 +                return false;
 +
 +            for (int i = 0; i < elements.size(); i++)
 +                if (lt.elements.compare(elements.get(i), v.elements.get(i)) != 0)
 +                    return false;
 +
 +            return true;
          }
+ 
+         public List<ByteBuffer> getElements()
+         {
+             return elements;
+         }
      }
  
-     /*
+     /**
       * Basically similar to a Value, but with some non-pure function (that need
       * to be evaluated at execution time) in it.
       *
@@@ -223,11 -216,10 +231,10 @@@
              assert receiver.type instanceof ListType;
          }
  
 -        public Value bind(List<ByteBuffer> values) throws InvalidRequestException
 +        public Value bind(QueryOptions options) throws InvalidRequestException
          {
 -            ByteBuffer value = values.get(bindIndex);
 -            return value == null ? null : Value.fromSerialized(value, (ListType)receiver.type);
 +            ByteBuffer value = options.getValues().get(bindIndex);
 +            return value == null ? null : Value.fromSerialized(value, (ListType)receiver.type, options.getProtocolVersion());
- 
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Relation.java
index 2eeef1d,0f1366d..5318907
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@@ -17,78 -17,35 +17,47 @@@
   */
  package org.apache.cassandra.cql3;
  
- import java.util.ArrayList;
- import java.util.List;
 +
- /**
-  * Relations encapsulate the relationship between an entity of some kind, and
-  * a value (term). For example, <key> > "start" or "colname1" = "somevalue".
-  *
-  */
- public class Relation
- {
-     private final ColumnIdentifier entity;
-     private final Type relationType;
-     private final Term.Raw value;
-     private final List<Term.Raw> inValues;
-     public final boolean onToken;
+ public abstract class Relation {
  
-     // Will be null unless for tuple notations (#4851)
-     public final ColumnIdentifier previousInTuple;
+     protected Type relationType;
  
      public static enum Type
      {
 -        EQ, LT, LTE, GTE, GT, IN;
 +        EQ, LT, LTE, GTE, GT, IN, CONTAINS, CONTAINS_KEY;
 +
 +        public boolean allowsIndexQuery()
 +        {
 +            switch (this)
 +            {
 +                case EQ:
 +                case CONTAINS:
 +                case CONTAINS_KEY:
 +                    return true;
 +                default:
 +                    return false;
 +            }
 +        }
-     }
- 
-     private Relation(ColumnIdentifier entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken, ColumnIdentifier previousInTuple)
-     {
-         this.entity = entity;
-         this.relationType = type;
-         this.value = value;
-         this.inValues = inValues;
-         this.onToken = onToken;
-         this.previousInTuple = previousInTuple;
-     }
  
-     /**
-      * Creates a new relation.
-      *
-      * @param entity the kind of relation this is; what the term is being compared to.
-      * @param type the type that describes how this entity relates to the value.
-      * @param value the value being compared.
-      */
-     public Relation(ColumnIdentifier entity, Type type, Term.Raw value)
-     {
-         this(entity, type, value, null, false, null);
-     }
- 
-     public Relation(ColumnIdentifier entity, Type type, Term.Raw value, boolean onToken)
-     {
-         this(entity, type, value, null, onToken, null);
-     }
- 
-     public Relation(ColumnIdentifier entity, Type type, Term.Raw value, ColumnIdentifier previousInTuple)
-     {
-         this(entity, type, value, null, false, previousInTuple);
-     }
- 
-     public static Relation createInRelation(ColumnIdentifier entity)
-     {
-         return new Relation(entity, Type.IN, null, new ArrayList<Term.Raw>(), false, null);
+         @Override
+         public String toString()
+         {
+             switch (this)
+             {
+                 case EQ:
+                     return "=";
+                 case LT:
+                     return "<";
+                 case LTE:
+                     return "<=";
+                 case GT:
+                     return ">";
+                 case GTE:
+                     return ">=";
 -                case IN:
 -                    return "IN";
+                 default:
+                     return this.name();
+             }
+         }
      }
  
      public Type operator()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/Term.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Term.java
index de866e1,96b4b71..e5206c8
--- a/src/java/org/apache/cassandra/cql3/Term.java
+++ b/src/java/org/apache/cassandra/cql3/Term.java
@@@ -88,9 -88,14 +88,14 @@@ public interface Ter
           * case this RawTerm describe a list index or a map key, etc...
           * @return the prepared term.
           */
 -        public Term prepare(ColumnSpecification receiver) throws InvalidRequestException;
 +        public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException;
      }
  
+     public interface MultiColumnRaw extends Raw
+     {
 -        public Term prepare(List<? extends ColumnSpecification> receiver) throws InvalidRequestException;
++        public Term prepare(String keyspace, List<? extends ColumnSpecification> receiver) throws InvalidRequestException;
+     }
+ 
      /**
       * A terminal term, one that can be reduced to a byte buffer directly.
       *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Tuples.java
index 0000000,9e86912..3aabbd3
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@@ -1,0 -1,349 +1,349 @@@
+ /*
+  * 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 org.apache.cassandra.db.marshal.AbstractType;
+ import org.apache.cassandra.db.marshal.CollectionType;
+ import org.apache.cassandra.db.marshal.ListType;
+ import org.apache.cassandra.db.marshal.TupleType;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.serializers.MarshalException;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.*;
+ 
+ /**
+  * Static helper methods and classes for tuples.
+  */
+ public class Tuples
+ {
+     private static final Logger logger = LoggerFactory.getLogger(Tuples.class);
+ 
+     /**
+      * A raw, literal tuple.  When prepared, this will become a Tuples.Value or Tuples.DelayedValue, depending
+      * on whether the tuple holds NonTerminals.
+      */
+     public static class Literal implements Term.MultiColumnRaw
+     {
+         private final List<Term.Raw> elements;
+ 
+         public Literal(List<Term.Raw> elements)
+         {
+             this.elements = elements;
+         }
+ 
 -        public Term prepare(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
++        public Term prepare(String keyspace, List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+         {
+             if (elements.size() != receivers.size())
+                 throw new InvalidRequestException(String.format("Expected %d elements in value tuple, but got %d: %s", receivers.size(), elements.size(), this));
+ 
+             List<Term> values = new ArrayList<>(elements.size());
+             boolean allTerminal = true;
+             for (int i = 0; i < elements.size(); i++)
+             {
 -                Term t = elements.get(i).prepare(receivers.get(i));
++                Term t = elements.get(i).prepare(keyspace, receivers.get(i));
+                 if (t instanceof Term.NonTerminal)
+                     allTerminal = false;
+ 
+                 values.add(t);
+             }
+             DelayedValue value = new DelayedValue(values);
 -            return allTerminal ? value.bind(Collections.<ByteBuffer>emptyList()) : value;
++            return allTerminal ? value.bind(QueryOptions.DEFAULT) : value;
+         }
+ 
 -        public Term prepare(ColumnSpecification receiver)
++        public Term prepare(String keyspace, ColumnSpecification receiver)
+         {
+             throw new AssertionError("Tuples.Literal instances require a list of receivers for prepare()");
+         }
+ 
 -        public boolean isAssignableTo(ColumnSpecification receiver)
++        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+         {
+             // tuples shouldn't be assignable to anything right now
+             return false;
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return tupleToString(elements);
+         }
+     }
+ 
+     /**
+      * A tuple of terminal values (e.g (123, 'abc')).
+      */
+     public static class Value extends Term.MultiItemTerminal
+     {
+         public final ByteBuffer[] elements;
+ 
+         public Value(ByteBuffer[] elements)
+         {
+             this.elements = elements;
+         }
+ 
+         public static Value fromSerialized(ByteBuffer bytes, TupleType type)
+         {
+             return new Value(type.split(bytes));
+         }
+ 
 -        public ByteBuffer get()
++        public ByteBuffer get(QueryOptions options)
+         {
+             throw new UnsupportedOperationException();
+         }
+ 
+         public List<ByteBuffer> getElements()
+         {
+             return Arrays.asList(elements);
+         }
+     }
+ 
+     /**
+      * Similar to Value, but contains at least one NonTerminal, such as a non-pure functions or bind marker.
+      */
+     public static class DelayedValue extends Term.NonTerminal
+     {
+         public final List<Term> elements;
+ 
+         public DelayedValue(List<Term> elements)
+         {
+             this.elements = elements;
+         }
+ 
+         public boolean containsBindMarker()
+         {
+             for (Term term : elements)
+                 if (term.containsBindMarker())
+                     return true;
+ 
+             return false;
+         }
+ 
+         public void collectMarkerSpecification(VariableSpecifications boundNames)
+         {
+             for (Term term : elements)
+                 term.collectMarkerSpecification(boundNames);
+         }
+ 
 -        public Value bind(List<ByteBuffer> values) throws InvalidRequestException
++        public Value bind(QueryOptions options) throws InvalidRequestException
+         {
+             ByteBuffer[] buffers = new ByteBuffer[elements.size()];
+             for (int i=0; i < elements.size(); i++)
+             {
 -                ByteBuffer bytes = elements.get(i).bindAndGet(values);
++                ByteBuffer bytes = elements.get(i).bindAndGet(options);
+                 if (bytes == null)
+                     throw new InvalidRequestException("Tuples may not contain null values");
+ 
 -                buffers[i] = elements.get(i).bindAndGet(values);
++                buffers[i] = elements.get(i).bindAndGet(options);
+             }
+             return new Value(buffers);
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return tupleToString(elements);
+         }
+     }
+ 
+     /**
+      * A terminal value for a list of IN values that are tuples. For example: "SELECT ... WHERE (a, b, c) IN ?"
+      * This is similar to Lists.Value, but allows us to keep components of the tuples in the list separate.
+      */
+     public static class InValue extends Term.Terminal
+     {
+         List<List<ByteBuffer>> elements;
+ 
+         public InValue(List<List<ByteBuffer>> items)
+         {
+             this.elements = items;
+         }
+ 
+         public static InValue fromSerialized(ByteBuffer value, ListType type) throws InvalidRequestException
+         {
+             try
+             {
+                 // Collections have this small hack that validate cannot be called on a serialized object,
+                 // but compose does the validation (so we're fine).
+                 List<?> l = (List<?>)type.compose(value);
+ 
+                 assert type.elements instanceof TupleType;
+                 TupleType tupleType = (TupleType) type.elements;
+ 
+                 // type.split(bytes)
+                 List<List<ByteBuffer>> elements = new ArrayList<>(l.size());
+                 for (Object element : l)
+                     elements.add(Arrays.asList(tupleType.split(type.elements.decompose(element))));
+                 return new InValue(elements);
+             }
+             catch (MarshalException e)
+             {
+                 throw new InvalidRequestException(e.getMessage());
+             }
+         }
+ 
 -        public ByteBuffer get()
++        public ByteBuffer get(QueryOptions options)
+         {
+             throw new UnsupportedOperationException();
+         }
+ 
+         public List<List<ByteBuffer>> getSplitValues()
+         {
+             return elements;
+         }
+     }
+ 
+     /**
+      * A raw placeholder for a tuple of values for different multiple columns, each of which may have a different type.
+      * For example, "SELECT ... WHERE (col1, col2) > ?".
+      */
+     public static class Raw extends AbstractMarker.Raw implements Term.MultiColumnRaw
+     {
+         public Raw(int bindIndex)
+         {
+             super(bindIndex);
+         }
+ 
+         private static ColumnSpecification makeReceiver(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+         {
+             List<AbstractType<?>> types = new ArrayList<>(receivers.size());
+             StringBuilder inName = new StringBuilder("(");
+             for (int i = 0; i < receivers.size(); i++)
+             {
+                 ColumnSpecification receiver = receivers.get(i);
+                 inName.append(receiver.name);
+                 if (i < receivers.size() - 1)
+                     inName.append(",");
+                 types.add(receiver.type);
+             }
+             inName.append(')');
+ 
+             ColumnIdentifier identifier = new ColumnIdentifier(inName.toString(), true);
+             TupleType type = new TupleType(types);
+             return new ColumnSpecification(receivers.get(0).ksName, receivers.get(0).cfName, identifier, type);
+         }
+ 
 -        public AbstractMarker prepare(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
++        public AbstractMarker prepare(String keyspace, List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+         {
+             return new Tuples.Marker(bindIndex, makeReceiver(receivers));
+         }
+ 
+         @Override
 -        public AbstractMarker prepare(ColumnSpecification receiver)
++        public AbstractMarker prepare(String keyspace, ColumnSpecification receiver)
+         {
+             throw new AssertionError("Tuples.Raw.prepare() requires a list of receivers");
+         }
+     }
+ 
+     /**
+      * A raw marker for an IN list of tuples, like "SELECT ... WHERE (a, b, c) IN ?"
+      */
+     public static class INRaw extends AbstractMarker.Raw
+     {
+         public INRaw(int bindIndex)
+         {
+             super(bindIndex);
+         }
+ 
+         private static ColumnSpecification makeInReceiver(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+         {
+             List<AbstractType<?>> types = new ArrayList<>(receivers.size());
+             StringBuilder inName = new StringBuilder("in(");
+             for (int i = 0; i < receivers.size(); i++)
+             {
+                 ColumnSpecification receiver = receivers.get(i);
+                 inName.append(receiver.name);
+                 if (i < receivers.size() - 1)
+                     inName.append(",");
+ 
+                 if (receiver.type instanceof CollectionType)
+                     throw new InvalidRequestException("Collection columns do not support IN relations");
+                 types.add(receiver.type);
+             }
+             inName.append(')');
+ 
+             ColumnIdentifier identifier = new ColumnIdentifier(inName.toString(), true);
+             TupleType type = new TupleType(types);
+             return new ColumnSpecification(receivers.get(0).ksName, receivers.get(0).cfName, identifier, ListType.getInstance(type));
+         }
+ 
 -        public AbstractMarker prepare(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
++        public AbstractMarker prepare(String keyspace, List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+         {
+             return new InMarker(bindIndex, makeInReceiver(receivers));
+         }
+ 
+         @Override
 -        public AbstractMarker prepare(ColumnSpecification receiver)
++        public AbstractMarker prepare(String keyspace, ColumnSpecification receiver)
+         {
+             throw new AssertionError("Tuples.INRaw.prepare() requires a list of receivers");
+         }
+     }
+ 
+     /**
+      * Represents a marker for a single tuple, like "SELECT ... WHERE (a, b, c) > ?"
+      */
+     public static class Marker extends AbstractMarker
+     {
+         public Marker(int bindIndex, ColumnSpecification receiver)
+         {
+             super(bindIndex, receiver);
+         }
+ 
 -        public Value bind(List<ByteBuffer> values) throws InvalidRequestException
++        public Value bind(QueryOptions options) throws InvalidRequestException
+         {
 -            ByteBuffer value = values.get(bindIndex);
++            ByteBuffer value = options.getValues().get(bindIndex);
+             if (value == null)
+                 return null;
+ 
+             return value == null ? null : Value.fromSerialized(value, (TupleType)receiver.type);
+         }
+     }
+ 
+     /**
+      * Represents a marker for a set of IN values that are tuples, like "SELECT ... WHERE (a, b, c) IN ?"
+      */
+     public static class InMarker extends AbstractMarker
+     {
+         protected InMarker(int bindIndex, ColumnSpecification receiver)
+         {
+             super(bindIndex, receiver);
+             assert receiver.type instanceof ListType;
+         }
+ 
 -        public InValue bind(List<ByteBuffer> values) throws InvalidRequestException
++        public InValue bind(QueryOptions options) throws InvalidRequestException
+         {
 -            ByteBuffer value = values.get(bindIndex);
++            ByteBuffer value = options.getValues().get(bindIndex);
+             return value == null ? null : InValue.fromSerialized(value, (ListType)receiver.type);
+         }
+     }
+ 
+     public static String tupleToString(List<?> items)
+     {
+ 
+         StringBuilder sb = new StringBuilder("(");
+         for (int i = 0; i < items.size(); i++)
+         {
+             sb.append(items.get(i));
+             if (i < items.size() - 1)
+                 sb.append(", ");
+         }
+         sb.append(')');
+         return sb.toString();
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index ad88eaf,11aa0b1..621006b
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -29,10 -31,10 +29,12 @@@ import org.apache.cassandra.config.CFMe
  import org.apache.cassandra.config.ColumnDefinition;
  import org.apache.cassandra.cql3.*;
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.composites.CBuilder;
 +import org.apache.cassandra.db.composites.Composite;
  import org.apache.cassandra.db.filter.ColumnSlice;
  import org.apache.cassandra.db.filter.SliceQueryFilter;
+ import org.apache.cassandra.db.marshal.CompositeType;
+ import org.apache.cassandra.db.marshal.UTF8Type;
  import org.apache.cassandra.db.marshal.BooleanType;
  import org.apache.cassandra.exceptions.*;
  import org.apache.cassandra.service.ClientState;
@@@ -218,46 -237,54 +220,53 @@@ public abstract class ModificationState
          return ifExists;
      }
  
 -    private void addKeyValues(CFDefinition.Name name, Restriction values) throws InvalidRequestException
 +    private void addKeyValues(ColumnDefinition def, Restriction values) throws InvalidRequestException
      {
 -        if (name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS)
 +        if (def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN)
              hasNoClusteringColumns = false;
 -        if (processedKeys.put(name.name, values) != null)
 -            throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", name.name));
 +        if (processedKeys.put(def.name, values) != null)
 +            throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", def.name));
      }
  
 -    public void addKeyValue(CFDefinition.Name name, Term value) throws InvalidRequestException
 +    public void addKeyValue(ColumnDefinition def, Term value) throws InvalidRequestException
      {
-         addKeyValues(def, new Restriction.EQ(value, false));
 -        addKeyValues(name, new SingleColumnRestriction.EQ(value, false));
++        addKeyValues(def, new SingleColumnRestriction.EQ(value, false));
      }
  
      public void processWhereClause(List<Relation> whereClause, VariableSpecifications names) throws InvalidRequestException
      {
-         for (Relation rel : whereClause)
 -        CFDefinition cfDef = cfm.getCfDef();
+         for (Relation relation : whereClause)
          {
 -            if (!(relation instanceof SingleColumnRelation))
++            if (relation.isMultiColumn())
+             {
+                 throw new InvalidRequestException(
+                         String.format("Multi-column relations cannot be used in WHERE clauses for modification statements: %s", relation));
+             }
+             SingleColumnRelation rel = (SingleColumnRelation) relation;
+ 
 -            CFDefinition.Name name = cfDef.get(rel.getEntity());
 -            if (name == null)
 +            ColumnDefinition def = cfm.getColumnDefinition(rel.getEntity());
 +            if (def == null)
                  throw new InvalidRequestException(String.format("Unknown key identifier %s", rel.getEntity()));
  
 -            switch (name.kind)
 +            switch (def.kind)
              {
 -                case KEY_ALIAS:
 -                case COLUMN_ALIAS:
 +                case PARTITION_KEY:
 +                case CLUSTERING_COLUMN:
                      Restriction restriction;
  
                      if (rel.operator() == Relation.Type.EQ)
                      {
 -                        Term t = rel.getValue().prepare(name);
 +                        Term t = rel.getValue().prepare(keyspace(), def);
                          t.collectMarkerSpecification(names);
-                         restriction = new Restriction.EQ(t, false);
+                         restriction = new SingleColumnRestriction.EQ(t, false);
                      }
 -                    else if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS && rel.operator() == Relation.Type.IN)
 +                    else if (def.kind == ColumnDefinition.Kind.PARTITION_KEY && rel.operator() == Relation.Type.IN)
                      {
                          if (rel.getValue() != null)
                          {
 -                            Term t = rel.getValue().prepare(name);
 +                            Term t = rel.getValue().prepare(keyspace(), def);
                              t.collectMarkerSpecification(names);
-                             restriction = Restriction.IN.create(t);
+                             restriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
                          }
                          else
                          {
@@@ -638,9 -689,9 +647,8 @@@
      /**
       * Convert statement into a list of mutations to apply on the server
       *
 -     * @param variables value for prepared statement markers
 +     * @param options value for prepared statement markers
       * @param local if true, any requests (for collections) performed by getMutation should be done locally only.
--     * @param cl the consistency to use for the potential reads involved in generating the mutations (for lists set/delete operations)
       * @param now the current timestamp in microseconds to use if no timestamp is user provided.
       *
       * @return list of the mutations

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
index 0000000,f643684..96cb905
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
@@@ -1,0 -1,135 +1,137 @@@
+ /*
+  * 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.statements;
+ 
+ import org.apache.cassandra.cql3.AbstractMarker;
++import org.apache.cassandra.cql3.QueryOptions;
+ import org.apache.cassandra.cql3.Term;
+ import org.apache.cassandra.cql3.Tuples;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.ArrayList;
+ import java.util.List;
+ 
+ public interface MultiColumnRestriction extends Restriction
+ {
+     public static class EQ extends SingleColumnRestriction.EQ implements MultiColumnRestriction
+     {
+         public EQ(Term value, boolean onToken)
+         {
+             super(value, onToken);
+         }
+ 
+         public boolean isMultiColumn()
+         {
+             return true;
+         }
+ 
 -        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
++        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+         {
 -            Tuples.Value t = (Tuples.Value)value.bind(variables);
++            Tuples.Value t = (Tuples.Value)value.bind(options);
+             return t.getElements();
+         }
+     }
+ 
+     public interface IN extends MultiColumnRestriction
+     {
 -        public List<List<ByteBuffer>> splitValues(List<ByteBuffer> variables) throws InvalidRequestException;
++        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
+     }
+ 
+     /**
+      * An IN restriction that has a set of terms for in values.
+      * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
+      */
+     public static class InWithValues extends SingleColumnRestriction.InWithValues implements MultiColumnRestriction.IN
+     {
+         public InWithValues(List<Term> values)
+         {
+             super(values);
+         }
+ 
+         public boolean isMultiColumn()
+         {
+             return true;
+         }
+ 
 -        public List<List<ByteBuffer>> splitValues(List<ByteBuffer> variables) throws InvalidRequestException
++        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
+         {
+             List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
+             for (Term value : values)
+             {
 -                Term.MultiItemTerminal term = (Term.MultiItemTerminal)value.bind(variables);
++                Term.MultiItemTerminal term = (Term.MultiItemTerminal)value.bind(options);
+                 buffers.add(term.getElements());
+             }
+             return buffers;
+         }
+     }
+ 
+     /**
+      * An IN restriction that uses a single marker for a set of IN values that are tuples.
+      * For example: "SELECT ... WHERE (a, b, c) IN ?"
+      */
+     public static class InWithMarker extends SingleColumnRestriction.InWithMarker implements MultiColumnRestriction.IN
+     {
+         public InWithMarker(AbstractMarker marker)
+         {
+             super(marker);
+         }
+ 
+         public boolean isMultiColumn()
+         {
+             return true;
+         }
+ 
 -        public List<List<ByteBuffer>> splitValues(List<ByteBuffer> variables) throws InvalidRequestException
++        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
+         {
 -            Tuples.InValue inValue = ((Tuples.InMarker) marker).bind(variables);
++            Tuples.InMarker inMarker = (Tuples.InMarker)marker;
++            Tuples.InValue inValue = inMarker.bind(options);
+             if (inValue == null)
+                 throw new InvalidRequestException("Invalid null value for IN restriction");
+             return inValue.getSplitValues();
+         }
+     }
+ 
+     public static class Slice extends SingleColumnRestriction.Slice implements MultiColumnRestriction
+     {
+         public Slice(boolean onToken)
+         {
+             super(onToken);
+         }
+ 
+         public boolean isMultiColumn()
+         {
+             return true;
+         }
+ 
 -        public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
++        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
+         {
+             throw new UnsupportedOperationException("Multicolumn slice restrictions do not support bound()");
+         }
+ 
+         /**
+          * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
+          * ByteBuffer to represent the entire bound.
+          */
 -        public List<ByteBuffer> componentBounds(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
++        public List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
+         {
 -            Tuples.Value value = (Tuples.Value)bounds[b.idx].bind(variables);
++            Tuples.Value value = (Tuples.Value)bounds[b.idx].bind(options);
+             return value.getElements();
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/statements/Restriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/Restriction.java
index 4fd02c1,3d33bde..c529a38
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@@ -18,14 -18,10 +18,10 @@@
  package org.apache.cassandra.cql3.statements;
  
  import java.nio.ByteBuffer;
- import java.util.ArrayList;
- import java.util.Collections;
  import java.util.List;
  
- import com.google.common.base.Objects;
- 
  import org.apache.cassandra.exceptions.InvalidRequestException;
 -import org.apache.cassandra.thrift.IndexOperator;
 +import org.apache.cassandra.db.IndexExpression;
  import org.apache.cassandra.cql3.*;
  
  /**
@@@ -39,398 -35,34 +35,35 @@@ public interface Restrictio
      public boolean isSlice();
      public boolean isEQ();
      public boolean isIN();
 +    public boolean isContains();
+     public boolean isMultiColumn();
  
 -    // Only supported for EQ and IN, but it's convenient to have here
 -    public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException;
 +    // Not supported by Slice, but it's convenient to have here
 +    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
  
-     public static class EQ implements Restriction
-     {
-         private final Term value;
-         private final boolean onToken;
- 
-         public EQ(Term value, boolean onToken)
-         {
-             this.value = value;
-             this.onToken = onToken;
-         }
- 
-         public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-         {
-             return Collections.singletonList(value.bindAndGet(options));
-         }
- 
-         public boolean isSlice()
-         {
-             return false;
-         }
- 
-         public boolean isEQ()
-         {
-             return true;
-         }
- 
-         public boolean isIN()
-         {
-             return false;
-         }
- 
-         public boolean isContains()
-         {
-             return false;
-         }
+     public static interface EQ extends Restriction {}
  
-         public boolean isOnToken()
-         {
-             return onToken;
-         }
- 
-         @Override
-         public String toString()
-         {
-             return String.format("EQ(%s)%s", value, onToken ? "*" : "");
-         }
-     }
- 
-     public static abstract class IN implements Restriction
+     public static interface IN extends Restriction
      {
-         public static IN create(List<Term> values)
-         {
-             return new WithValues(values);
-         }
- 
-         public static IN create(Term value)
-         {
-             assert value instanceof Lists.Marker; // we shouldn't have got there otherwise
-             return new WithMarker((Lists.Marker)value);
-         }
- 
-         public boolean isSlice()
-         {
-             return false;
-         }
- 
-         public boolean isEQ()
-         {
-             return false;
-         }
- 
-         public boolean isContains()
-         {
-             return false;
-         }
- 
-         public boolean isIN()
-         {
-             return true;
-         }
- 
-         // Used when we need to know if it's a IN with just one value before we have
-         // the bind variables. This is ugly and only there for backward compatiblity
-         // because we used to treate IN with 1 value like an EQ and need to preserve
-         // this behavior.
-         public abstract boolean canHaveOnlyOneValue();
- 
-         public boolean isOnToken()
-         {
-             return false;
-         }
- 
-         private static class WithValues extends IN
-         {
-             private final List<Term> values;
- 
-             private WithValues(List<Term> values)
-             {
-                 this.values = values;
-             }
- 
-             public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-             {
-                 List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(values.size());
-                 for (Term value : values)
-                     buffers.add(value.bindAndGet(options));
-                 return buffers;
-             }
- 
-             public boolean canHaveOnlyOneValue()
-             {
-                 return values.size() == 1;
-             }
- 
-             @Override
-             public String toString()
-             {
-                 return String.format("IN(%s)", values);
-             }
-         }
- 
-         private static class WithMarker extends IN
-         {
-             private final Lists.Marker marker;
- 
-             private WithMarker(Lists.Marker marker)
-             {
-                 this.marker = marker;
-             }
- 
-             public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-             {
-                 Lists.Value lval = marker.bind(options);
-                 if (lval == null)
-                     throw new InvalidRequestException("Invalid null value for IN restriction");
-                 return lval.elements;
-             }
- 
-             public boolean canHaveOnlyOneValue()
-             {
-                 return false;
-             }
- 
-             @Override
-             public String toString()
-             {
-                 return "IN ?";
-             }
-         }
+         public boolean canHaveOnlyOneValue();
      }
  
-     public static class Slice implements Restriction
+     public static interface Slice extends Restriction
      {
-         private final Term[] bounds;
-         private final boolean[] boundInclusive;
-         private final boolean onToken;
- 
-         // The name of the column that was preceding this one if the tuple notation of #4851 was used
-         // (note: if it is set for both bound, we'll validate both have the same previous value, but we
-         // still need to distinguish if it's set or not for both bound)
-         private final ColumnIdentifier[] previous;
- 
-         public Slice(boolean onToken)
-         {
-             this.bounds = new Term[2];
-             this.boundInclusive = new boolean[2];
-             this.previous = new ColumnIdentifier[2];
-             this.onToken = onToken;
-         }
- 
-         public boolean isSlice()
-         {
-             return true;
-         }
- 
-         public boolean isEQ()
-         {
-             return false;
-         }
- 
-         public boolean isIN()
-         {
-             return false;
-         }
- 
-         public boolean isContains()
-         {
-             return false;
-         }
- 
-         public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-         {
-             throw new UnsupportedOperationException();
-         }
- 
-         public boolean isOnToken()
-         {
-             return onToken;
-         }
- 
-         public boolean hasBound(Bound b)
-         {
-             return bounds[b.idx] != null;
-         }
- 
-         public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
-         {
-             return bounds[b.idx].bindAndGet(options);
-         }
- 
-         public boolean isInclusive(Bound b)
-         {
-             return bounds[b.idx] == null || boundInclusive[b.idx];
-         }
- 
-         public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound)
-         {
-             switch (eocBound)
-             {
-                 case START:
-                     return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT;
-                 case END:
-                     return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT;
-             }
-             throw new AssertionError();
-         }
- 
-         public IndexExpression.Operator getIndexOperator(Bound b)
-         {
-             switch (b)
-             {
-                 case START:
-                     return boundInclusive[b.idx] ? IndexExpression.Operator.GTE : IndexExpression.Operator.GT;
-                 case END:
-                     return boundInclusive[b.idx] ? IndexExpression.Operator.LTE : IndexExpression.Operator.LT;
-             }
-             throw new AssertionError();
-         }
- 
-         public void setBound(ColumnIdentifier name, Relation.Type type, Term t, ColumnIdentifier previousName) throws InvalidRequestException
-         {
-             Bound b;
-             boolean inclusive;
-             switch (type)
-             {
-                 case GT:
-                     b = Bound.START;
-                     inclusive = false;
-                     break;
-                 case GTE:
-                     b = Bound.START;
-                     inclusive = true;
-                     break;
-                 case LT:
-                     b = Bound.END;
-                     inclusive = false;
-                     break;
-                 case LTE:
-                     b = Bound.END;
-                     inclusive = true;
-                     break;
-                 default:
-                     throw new AssertionError();
-             }
- 
-             if (bounds[b.idx] != null)
-                 throw new InvalidRequestException(String.format("Invalid restrictions found on %s", name));
- 
-             bounds[b.idx] = t;
-             boundInclusive[b.idx] = inclusive;
- 
-             // If a bound is part of a tuple notation (#4851), the other bound must either also be or must not be set at all,
-             // and this even if there is a 2ndary index (it's not supported by the 2ndary code). And it's easier to validate
-             // this here so we do.
-             Bound reverse = Bound.reverse(b);
-             if (hasBound(reverse) && !(Objects.equal(previousName, previous[reverse.idx])))
-                 throw new InvalidRequestException(String.format("Clustering column %s cannot be restricted both inside a tuple notation and outside it", name));
- 
-             previous[b.idx] = previousName;
-         }
- 
-         public boolean isPartOfTuple()
-         {
-             return previous[Bound.START.idx] != null || previous[Bound.END.idx] != null;
-         }
- 
-         @Override
-         public String toString()
-         {
-             return String.format("SLICE(%s %s, %s %s)%s", boundInclusive[0] ? ">=" : ">",
-                                                           bounds[0],
-                                                           boundInclusive[1] ? "<=" : "<",
-                                                           bounds[1],
-                                                           onToken ? "*" : "");
-         }
-     }
- 
-     // This holds both CONTAINS and CONTAINS_KEY restriction because we might want to have both of them.
-     public static class Contains implements Restriction
-     {
-         private List<Term> values; // for CONTAINS
-         private List<Term> keys;   // for CONTAINS_KEY
- 
-         public boolean hasContains()
-         {
-             return values != null;
-         }
- 
-         public boolean hasContainsKey()
-         {
-             return keys != null;
-         }
- 
-         public void add(Term t, boolean isKey)
-         {
-             if (isKey)
-                 addKey(t);
-             else
-                 addValue(t);
-         }
- 
-         public void addValue(Term t)
-         {
-             if (values == null)
-                 values = new ArrayList<>();
-             values.add(t);
-         }
- 
-         public void addKey(Term t)
-         {
-             if (keys == null)
-                 keys = new ArrayList<>();
-             keys.add(t);
-         }
- 
-         public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-         {
-             if (values == null)
-                 return Collections.emptyList();
- 
-             List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(values.size());
-             for (Term value : values)
-                 buffers.add(value.bindAndGet(options));
-             return buffers;
-         }
- 
-         public List<ByteBuffer> keys(QueryOptions options) throws InvalidRequestException
-         {
-             if (keys == null)
-                 return Collections.emptyList();
- 
-             List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(keys.size());
-             for (Term value : keys)
-                 buffers.add(value.bindAndGet(options));
-             return buffers;
-         }
- 
-         public boolean isSlice()
-         {
-             return false;
-         }
 -        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException;
++        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
  
-         public boolean isEQ()
-         {
-             return false;
-         }
+         /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
+         public boolean hasBound(Bound b);
  
-         public boolean isIN()
-         {
-             return false;
-         }
 -        public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException;
++        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException;
  
-         public boolean isContains()
-         {
-             return true;
-         }
+         /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
+         public boolean isInclusive(Bound b);
  
-         public boolean isOnToken()
-         {
-             return false;
-         }
+         public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound);
  
 -        public IndexOperator getIndexOperator(Bound b);
++        public IndexExpression.Operator getIndexOperator(Bound b);
  
-         @Override
-         public String toString()
-         {
-             return String.format("CONTAINS(values=%s, keys=%s)", values, keys);
-         }
 -        public void setBound(Relation.Type type, Term t) throws InvalidRequestException;
++        public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException;
      }
  }


[5/7] Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 420f475,e339ccb..1f9688a
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -69,12 -70,17 +69,17 @@@ public class SelectStatement implement
      private final Selection selection;
      private final Term limit;
  
+     /** Restrictions on partitioning columns */
      private final Restriction[] keyRestrictions;
+ 
+     /** Restrictions on clustering columns */
      private final Restriction[] columnRestrictions;
+ 
+     /** Restrictions on non-primary key columns (i.e. secondary index restrictions) */
 -    private final Map<CFDefinition.Name, Restriction> metadataRestrictions = new HashMap<CFDefinition.Name, Restriction>();
 +    private final Map<ColumnIdentifier, Restriction> metadataRestrictions = new HashMap<ColumnIdentifier, Restriction>();
  
 -    // The name of all restricted names not covered by the key or index filter
 -    private final Set<CFDefinition.Name> restrictedNames = new HashSet<CFDefinition.Name>();
 +    // All restricted columns not covered by the key or index filter
 +    private final Set<ColumnDefinition> restrictedColumns = new HashSet<ColumnDefinition>();
      private Restriction.Slice sliceRestriction;
  
      private boolean isReversed;
@@@ -327,9 -335,9 +332,9 @@@
          if (keys.isEmpty()) // in case of IN () for (the last column of) the partition key.
              return null;
  
-         List<ReadCommand> commands = new ArrayList<ReadCommand>(keys.size());
+         List<ReadCommand> commands = new ArrayList<>(keys.size());
  
 -        IDiskAtomFilter filter = makeFilter(variables, limit);
 +        IDiskAtomFilter filter = makeFilter(options, limit);
          if (filter == null)
              return null;
  
@@@ -765,15 -794,29 +777,30 @@@
          return false;
      }
  
 -    private List<ByteBuffer> buildBound(Bound bound,
 -                                        Collection<CFDefinition.Name> names,
 -                                        Restriction[] restrictions,
 -                                        boolean isReversed,
 -                                        ColumnNameBuilder builder,
 -                                        List<ByteBuffer> variables) throws InvalidRequestException
 +    private static List<Composite> buildBound(Bound bound,
 +                                              Collection<ColumnDefinition> defs,
 +                                              Restriction[] restrictions,
 +                                              boolean isReversed,
 +                                              CType type,
 +                                              QueryOptions options) throws InvalidRequestException
      {
 +        CBuilder builder = type.builder();
  
+         // check the first restriction to see if we're dealing with a multi-column restriction
 -        if (!names.isEmpty())
++        if (!defs.isEmpty())
+         {
+             Restriction firstRestriction = restrictions[0];
+             if (firstRestriction != null && firstRestriction.isMultiColumn())
+             {
+                 if (firstRestriction.isSlice())
 -                    return buildMultiColumnSliceBound(bound, names, (MultiColumnRestriction.Slice) firstRestriction, isReversed, builder, variables);
++                    return buildMultiColumnSliceBound(bound, defs, (MultiColumnRestriction.Slice) firstRestriction, isReversed, builder, options);
+                 else if (firstRestriction.isIN())
 -                    return buildMultiColumnInBound(bound, names, (MultiColumnRestriction.IN) firstRestriction, isReversed, builder, variables);
++                    return buildMultiColumnInBound(bound, defs, (MultiColumnRestriction.IN) firstRestriction, isReversed, builder, type, options);
+                 else
 -                    return buildMultiColumnEQBound(bound, (MultiColumnRestriction.EQ) firstRestriction, isReversed, builder, variables);
++                    return buildMultiColumnEQBound(bound, (MultiColumnRestriction.EQ) firstRestriction, isReversed, builder, options);
+             }
+         }
+ 
          // The end-of-component of composite doesn't depend on whether the
          // component type is reversed or not (i.e. the ReversedType is applied
          // to the component comparator but not to the end-of-component itself),
@@@ -793,47 -836,35 +820,34 @@@
                  // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix.
                  // For composites, if there was preceding component and we're computing the end, we must change the last component
                  // End-Of-Component, otherwise we would be selecting only one record.
 -                return Collections.singletonList(builder.componentCount() > 0 && eocBound == Bound.END
 -                                                 ? builder.buildAsEndOfRange()
 -                                                 : builder.build());
 +                Composite prefix = builder.build();
 +                return Collections.singletonList(!prefix.isEmpty() && eocBound == Bound.END ? prefix.end() : prefix);
              }
- 
              if (r.isSlice())
              {
-                 builder.add(getSliceValue(def, r, b, options));
 -                builder.add(getSliceValue(r, b, variables));
++                builder.add(getSliceValue(r, b, options));
                  Relation.Type relType = ((Restriction.Slice)r).getRelation(eocBound, b);
- 
-                 // We can have more non null restriction if the "scalar" notation was used for the bound (#4851).
-                 // In that case, we need to add them all, and end the cell name with the correct end-of-component.
-                 while (iter.hasNext())
-                 {
-                     def = iter.next();
-                     r = restrictions[def.position()];
-                     if (isNullRestriction(r, b))
-                         break;
- 
-                     builder.add(getSliceValue(def, r, b, options));
-                 }
 -                return Collections.singletonList(builder.buildForRelation(relType));
 +                return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
              }
              else
              {
 -                List<ByteBuffer> values = r.values(variables);
 +                List<ByteBuffer> values = r.values(options);
                  if (values.size() != 1)
                  {
-                     // IN query, we only support it on the clustering column
+                     // IN query, we only support it on the clustering columns
 -                    assert name.position == names.size() - 1;
 +                    assert def.position() == defs.size() - 1;
                      // The IN query might not have listed the values in comparator order, so we need to re-sort
                      // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
-                     TreeSet<Composite> s = new TreeSet<Composite>(isReversed ? type.reverseComparator() : type);
 -                    TreeSet<ByteBuffer> s = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
++                    TreeSet<Composite> s = new TreeSet<>(isReversed ? type.reverseComparator() : type);
                      for (ByteBuffer val : values)
                      {
                          if (val == null)
 -                            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
 -                        ColumnNameBuilder copy = builder.copy().add(val);
 +                            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
 +                        Composite prefix = builder.buildWith(val);
                          // See below for why this
 -                        s.add((b == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
 +                        s.add((b == Bound.END && builder.remainingCount() > 0) ? prefix.end() : prefix);
                      }
-                     return new ArrayList<Composite>(s);
+                     return new ArrayList<>(s);
                  }
  
                  ByteBuffer val = values.get(0);
@@@ -847,42 -878,96 +861,123 @@@
          // it would be harmless to do it. However, we use this method got the partition key too. And when a query
          // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
          // case using the eoc would be bad, since for the random partitioner we have no guarantee that
 -        // builder.buildAsEndOfRange() will sort after builder.build() (see #5240).
 -        return Collections.singletonList((bound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
 +        // prefix.end() will sort after prefix (see #5240).
 +        Composite prefix = builder.build();
 +        return Collections.singletonList(bound == Bound.END && builder.remainingCount() > 0 ? prefix.end() : prefix);
 +    }
 +
 +    private static Composite.EOC eocForRelation(Relation.Type op)
 +    {
 +        switch (op)
 +        {
 +            case LT:
 +                // < X => using startOf(X) as finish bound
 +                return Composite.EOC.START;
 +            case GT:
 +            case LTE:
 +                // > X => using endOf(X) as start bound
 +                // <= X => using endOf(X) as finish bound
 +                return Composite.EOC.END;
 +            default:
 +                // >= X => using X as start bound (could use START_OF too)
 +                // = X => using X
 +                return Composite.EOC.NONE;
 +        }
      }
  
 -    private List<ByteBuffer> buildMultiColumnSliceBound(Bound bound,
 -                                                        Collection<CFDefinition.Name> names,
 -                                                        MultiColumnRestriction.Slice slice,
 -                                                        boolean isReversed,
 -                                                        ColumnNameBuilder builder,
 -                                                        List<ByteBuffer> variables) throws InvalidRequestException
++    private static List<Composite> buildMultiColumnSliceBound(Bound bound,
++                                                              Collection<ColumnDefinition> defs,
++                                                              MultiColumnRestriction.Slice slice,
++                                                              boolean isReversed,
++                                                              CBuilder builder,
++                                                              QueryOptions options) throws InvalidRequestException
+     {
+         Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
+ 
 -        Iterator<CFDefinition.Name> iter = names.iterator();
 -        CFDefinition.Name firstName = iter.next();
++        Iterator<ColumnDefinition> iter = defs.iterator();
++        ColumnDefinition firstName = iter.next();
+         // A hack to preserve pre-6875 behavior for tuple-notation slices where the comparator mixes ASCENDING
+         // and DESCENDING orders.  This stores the bound for the first component; we will re-use it for all following
+         // components, even if they don't match the first component's reversal/non-reversal.  Note that this does *not*
+         // guarantee correct query results, it just preserves the previous behavior.
+         Bound firstComponentBound = isReversed == isReversedType(firstName) ? bound : Bound.reverse(bound);
+ 
+         if (!slice.hasBound(firstComponentBound))
 -            return Collections.singletonList(builder.componentCount() > 0 && eocBound == Bound.END
 -                    ? builder.buildAsEndOfRange()
 -                    : builder.build());
++        {
++            Composite prefix = builder.build();
++            return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
++                    ? prefix.end()
++                    : prefix);
++        }
+ 
 -        List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, variables);
 -        builder.add(vals.get(firstName.position));
++        List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, options);
++        builder.add(vals.get(firstName.position()));
+ 
+         while(iter.hasNext())
+         {
 -            CFDefinition.Name name = iter.next();
 -            if (name.position >= vals.size())
++            ColumnDefinition def = iter.next();
++            if (def.position() >= vals.size())
+                 break;
+ 
 -            builder.add(vals.get(name.position));
++            builder.add(vals.get(def.position()));
+         }
+         Relation.Type relType = slice.getRelation(eocBound, firstComponentBound);
 -        return Collections.singletonList(builder.buildForRelation(relType));
++        return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
+     }
+ 
 -    private List<ByteBuffer> buildMultiColumnInBound(Bound bound,
 -                                                     Collection<CFDefinition.Name> names,
 -                                                     MultiColumnRestriction.IN restriction,
 -                                                     boolean isReversed,
 -                                                     ColumnNameBuilder builder,
 -                                                     List<ByteBuffer> variables) throws InvalidRequestException
++    private static List<Composite> buildMultiColumnInBound(Bound bound,
++                                                           Collection<ColumnDefinition> defs,
++                                                           MultiColumnRestriction.IN restriction,
++                                                           boolean isReversed,
++                                                           CBuilder builder,
++                                                           CType type,
++                                                           QueryOptions options) throws InvalidRequestException
+     {
 -        List<List<ByteBuffer>> splitInValues = restriction.splitValues(variables);
++        List<List<ByteBuffer>> splitInValues = restriction.splitValues(options);
+ 
+         // The IN query might not have listed the values in comparator order, so we need to re-sort
+         // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
 -        TreeSet<ByteBuffer> inValues = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
 -        Iterator<CFDefinition.Name> iter = names.iterator();
++        TreeSet<Composite> inValues = new TreeSet<>(isReversed ? type.reverseComparator() : type);
++        Iterator<ColumnDefinition> iter = defs.iterator();
+         for (List<ByteBuffer> components : splitInValues)
+         {
 -            ColumnNameBuilder nameBuilder = builder.copy();
 -            for (ByteBuffer component : components)
 -                nameBuilder.add(component);
 -
++            Composite prefix = builder.buildWith(components);
+             Bound b = isReversed == isReversedType(iter.next()) ? bound : Bound.reverse(bound);
 -            inValues.add((bound == Bound.END && nameBuilder.remainingCount() > 0) ? nameBuilder.buildAsEndOfRange() : nameBuilder.build());
++            inValues.add(b == Bound.END && builder.remainingCount() - components.size() > 0
++                         ? prefix.end()
++                         : prefix);
+         }
+         return new ArrayList<>(inValues);
+     }
+ 
 -    private List<ByteBuffer> buildMultiColumnEQBound(Bound bound, MultiColumnRestriction.EQ restriction, boolean isReversed, ColumnNameBuilder builder, List<ByteBuffer> variables) throws InvalidRequestException
++    private static List<Composite> buildMultiColumnEQBound(Bound bound,
++                                                           MultiColumnRestriction.EQ restriction,
++                                                           boolean isReversed,
++                                                           CBuilder builder,
++                                                           QueryOptions options) throws InvalidRequestException
+     {
+         Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
 -        for (ByteBuffer component : restriction.values(variables))
++        for (ByteBuffer component : restriction.values(options))
+             builder.add(component);
+ 
 -        ByteBuffer result = builder.componentCount() > 0 && eocBound == Bound.END
 -                ? builder.buildAsEndOfRange()
 -                : builder.build();
 -        return Collections.singletonList(result);
++        Composite prefix = builder.build();
++        return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
++                                         ? prefix.end()
++                                         : prefix);
+     }
+ 
      private static boolean isNullRestriction(Restriction r, Bound b)
      {
          return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
      }
  
-     private static ByteBuffer getSliceValue(ColumnDefinition def, Restriction r, Bound b, QueryOptions options) throws InvalidRequestException
 -    private static ByteBuffer getSliceValue(Restriction r, Bound b, List<ByteBuffer> variables) throws InvalidRequestException
++    private static ByteBuffer getSliceValue(Restriction r, Bound b, QueryOptions options) throws InvalidRequestException
      {
          Restriction.Slice slice = (Restriction.Slice)r;
          assert slice.hasBound(b);
 -        ByteBuffer val = slice.bound(b, variables);
 +        ByteBuffer val = slice.bound(b, options);
          if (val == null)
-             throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
+             throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
          return val;
      }
  
@@@ -936,23 -1022,9 +1031,23 @@@
                      }
                  }
              }
 +            else if (restriction.isContains())
 +            {
-                 Restriction.Contains contains = (Restriction.Contains)restriction;
++                SingleColumnRestriction.Contains contains = (SingleColumnRestriction.Contains)restriction;
 +                for (ByteBuffer value : contains.values(options))
 +                {
 +                    validateIndexedValue(def, value);
 +                    expressions.add(new IndexExpression(def.name.bytes, IndexExpression.Operator.CONTAINS, value));
 +                }
 +                for (ByteBuffer key : contains.keys(options))
 +                {
 +                    validateIndexedValue(def, key);
 +                    expressions.add(new IndexExpression(def.name.bytes, IndexExpression.Operator.CONTAINS_KEY, key));
 +                }
 +            }
              else
              {
 -                List<ByteBuffer> values = restriction.values(variables);
 +                List<ByteBuffer> values = restriction.values(options);
  
                  if (values.size() != 1)
                      throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
@@@ -1218,28 -1352,23 +1313,20 @@@
          public ParsedStatement.Prepared prepare() throws InvalidRequestException
          {
              CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
--
-             VariableSpecifications names = getBoundVariables();
 -            CFDefinition cfDef = cfm.getCfDef();
 -
+             VariableSpecifications boundNames = getBoundVariables();
  
              // Select clause
              if (parameters.isCount && !selectClause.isEmpty())
                  throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
  
              Selection selection = selectClause.isEmpty()
 -                                ? Selection.wildcard(cfDef)
 -                                : Selection.fromSelectors(cfDef, selectClause);
 +                                ? Selection.wildcard(cfm)
 +                                : Selection.fromSelectors(cfm, selectClause);
  
              if (parameters.isDistinct)
 -                validateDistinctSelection(selection.getColumns(), cfDef.partitionKeys());
 +                validateDistinctSelection(selection.getColumns(), cfm.partitionKeyColumns());
  
-             Term prepLimit = null;
-             if (limit != null)
-             {
-                 prepLimit = limit.prepare(keyspace(), limitReceiver());
-                 prepLimit.collectMarkerSpecification(names);
-             }
- 
-             SelectStatement stmt = new SelectStatement(cfm, names.size(), parameters, selection, prepLimit);
 -            SelectStatement stmt = new SelectStatement(cfDef, boundNames.size(), parameters, selection, prepareLimit(boundNames));
++            SelectStatement stmt = new SelectStatement(cfm, boundNames.size(), parameters, selection, prepareLimit(boundNames));
  
              /*
               * WHERE clause. For a given entity, rules are:
@@@ -1251,51 -1380,318 +1338,338 @@@
               */
              boolean hasQueriableIndex = false;
              boolean hasQueriableClusteringColumnIndex = false;
-             for (Relation rel : whereClause)
+             for (Relation relation : whereClause)
              {
-                 ColumnDefinition def = cfm.getColumnDefinition(rel.getEntity());
-                 if (def == null)
+                 if (relation.isMultiColumn())
+                 {
+                     MultiColumnRelation rel = (MultiColumnRelation) relation;
 -                    List<CFDefinition.Name> names = new ArrayList<>(rel.getEntities().size());
++                    List<ColumnDefinition> names = new ArrayList<>(rel.getEntities().size());
+                     for (ColumnIdentifier entity : rel.getEntities())
+                     {
 -                        boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
++                        ColumnDefinition def = cfm.getColumnDefinition(entity);
++                        boolean[] queriable = processRelationEntity(stmt, relation, entity, def);
+                         hasQueriableIndex |= queriable[0];
+                         hasQueriableClusteringColumnIndex |= queriable[1];
 -                        names.add(cfDef.get(entity));
++                        names.add(def);
+                     }
+                     updateRestrictionsForRelation(stmt, names, rel, boundNames);
+                 }
+                 else
+                 {
+                     SingleColumnRelation rel = (SingleColumnRelation) relation;
 -                    boolean[] queriable = processRelationEntity(stmt, relation, rel.getEntity(), cfDef);
++                    ColumnIdentifier entity = rel.getEntity();
++                    ColumnDefinition def = cfm.getColumnDefinition(entity);
++                    boolean[] queriable = processRelationEntity(stmt, relation, entity, def);
+                     hasQueriableIndex |= queriable[0];
+                     hasQueriableClusteringColumnIndex |= queriable[1];
 -                    updateRestrictionsForRelation(stmt, cfDef.get(rel.getEntity()), rel, boundNames);
++                    updateRestrictionsForRelation(stmt, def, rel, boundNames);
+                 }
+             }
+ 
+              // At this point, the select statement if fully constructed, but we still have a few things to validate
 -            processPartitionKeyRestrictions(stmt, cfDef, hasQueriableIndex);
++            processPartitionKeyRestrictions(stmt, hasQueriableIndex, cfm);
+ 
+             // All (or none) of the partition key columns have been specified;
+             // hence there is no need to turn these restrictions into index expressions.
+             if (!stmt.usesSecondaryIndexing)
 -                stmt.restrictedNames.removeAll(cfDef.partitionKeys());
++                stmt.restrictedColumns.removeAll(cfm.partitionKeyColumns());
+ 
+             if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
+                 throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
+ 
 -            processColumnRestrictions(stmt, cfDef, hasQueriableIndex);
++            processColumnRestrictions(stmt, hasQueriableIndex, cfm);
+ 
+             // Covers indexes on the first clustering column (among others).
+             if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
+                 stmt.usesSecondaryIndexing = true;
+ 
+             if (!stmt.usesSecondaryIndexing)
 -                stmt.restrictedNames.removeAll(cfDef.clusteringColumns());
++                stmt.restrictedColumns.removeAll(cfm.clusteringColumns());
+ 
+             // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
+             // there is restrictions not covered by the PK.
+             if (!stmt.metadataRestrictions.isEmpty())
+             {
+                 if (!hasQueriableIndex)
+                     throw new InvalidRequestException("No indexed columns present in by-columns clause with Equal operator");
+                 stmt.usesSecondaryIndexing = true;
+             }
+ 
+             if (stmt.usesSecondaryIndexing)
+                 validateSecondaryIndexSelections(stmt);
+ 
+             if (!stmt.parameters.orderings.isEmpty())
 -                processOrderingClause(stmt, cfDef);
++                processOrderingClause(stmt, cfm);
+ 
+             checkNeedsFiltering(stmt);
+ 
+             return new ParsedStatement.Prepared(stmt, boundNames);
+         }
+ 
+         /** Returns a pair of (hasQueriableIndex, hasQueriableClusteringColumnIndex) */
 -        private boolean[] processRelationEntity(SelectStatement stmt, Relation relation, ColumnIdentifier entity, CFDefinition cfDef) throws InvalidRequestException
++        private boolean[] processRelationEntity(SelectStatement stmt, Relation relation, ColumnIdentifier entity, ColumnDefinition def) throws InvalidRequestException
+         {
 -            CFDefinition.Name name = cfDef.get(entity);
 -            if (name == null)
++            if (def == null)
+                 handleUnrecognizedEntity(entity, relation);
+ 
 -            stmt.restrictedNames.add(name);
 -            if (cfDef.cfm.getColumnDefinition(name.name.key).isIndexed() && relation.operator() == Relation.Type.EQ)
 -                return new boolean[]{true, name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS};
++            stmt.restrictedColumns.add(def);
++            if (def.isIndexed() && relation.operator().allowsIndexQuery())
++                return new boolean[]{true, def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN};
+             return new boolean[]{false, false};
+         }
+ 
+         /** Throws an InvalidRequestException for an unrecognized identifier in the WHERE clause */
+         private void handleUnrecognizedEntity(ColumnIdentifier entity, Relation relation) throws InvalidRequestException
+         {
+             if (containsAlias(entity))
+                 throw new InvalidRequestException(String.format("Aliases aren't allowed in the where clause ('%s')", relation));
+             else
+                 throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", entity, relation));
+         }
+ 
+         /** Returns a Term for the limit or null if no limit is set */
+         private Term prepareLimit(VariableSpecifications boundNames) throws InvalidRequestException
+         {
+             if (limit == null)
+                 return null;
+ 
 -            Term prepLimit = limit.prepare(limitReceiver());
++            Term prepLimit = limit.prepare(keyspace(), limitReceiver());
+             prepLimit.collectMarkerSpecification(boundNames);
+             return prepLimit;
+         }
+ 
 -        private void updateRestrictionsForRelation(SelectStatement stmt, List<CFDefinition.Name> names, MultiColumnRelation relation, VariableSpecifications boundNames) throws InvalidRequestException
++        private void updateRestrictionsForRelation(SelectStatement stmt, List<ColumnDefinition> defs, MultiColumnRelation relation, VariableSpecifications boundNames) throws InvalidRequestException
+         {
 -            List<CFDefinition.Name> restrictedColumns = new ArrayList<>();
 -            Set<CFDefinition.Name> seen = new HashSet<>();
++            List<ColumnDefinition> restrictedColumns = new ArrayList<>();
++            Set<ColumnDefinition> seen = new HashSet<>();
+ 
+             int previousPosition = -1;
 -            for (CFDefinition.Name name : names)
++            for (ColumnDefinition def : defs)
+             {
+                 // ensure multi-column restriction only applies to clustering columns
 -                if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
 -                    throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", name));
++                if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
++                    throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", def));
+ 
 -                if (seen.contains(name))
 -                    throw new InvalidRequestException(String.format("Column \"%s\" appeared twice in a relation: %s", name, relation));
 -                seen.add(name);
++                if (seen.contains(def))
++                    throw new InvalidRequestException(String.format("Column \"%s\" appeared twice in a relation: %s", def, relation));
++                seen.add(def);
+ 
+                 // check that no clustering columns were skipped
 -                if (name.position != previousPosition + 1)
++                if (def.position() != previousPosition + 1)
                  {
-                     if (containsAlias(rel.getEntity()))
-                         throw new InvalidRequestException(String.format("Aliases aren't allowed in where clause ('%s')", rel));
+                     if (previousPosition == -1)
+                         throw new InvalidRequestException(String.format(
+                                 "Clustering columns may not be skipped in multi-column relations. " +
+                                 "They should appear in the PRIMARY KEY order. Got %s", relation));
                      else
-                         throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", rel.getEntity(), rel));
+                         throw new InvalidRequestException(String.format(
+                                 "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
                  }
+                 previousPosition++;
  
-                 stmt.restrictedColumns.add(def);
-                 if (def.isIndexed() && rel.operator().allowsIndexQuery())
 -                Restriction existing = getExistingRestriction(stmt, name);
++                Restriction existing = getExistingRestriction(stmt, def);
+                 Relation.Type operator = relation.operator();
+                 if (existing != null)
                  {
-                     hasQueriableIndex = true;
-                     if (def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN)
-                         hasQueriableClusteringColumnIndex = true;
+                     if (operator == Relation.Type.EQ || operator == Relation.Type.IN)
 -                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation", name, relation.operator()));
++                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation", def, relation.operator()));
+                     else if (!existing.isSlice())
 -                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", name));
++                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", def));
                  }
 -                restrictedColumns.add(name);
++                restrictedColumns.add(def);
+             }
  
-                 switch (def.kind)
 -            boolean onToken = false;
 -
+             switch (relation.operator())
+             {
+                 case EQ:
                  {
-                     case PARTITION_KEY:
-                         stmt.keyRestrictions[def.position()] = updateRestriction(cfm, def, stmt.keyRestrictions[def.position()], rel, names);
-                         break;
-                     case CLUSTERING_COLUMN:
-                         stmt.columnRestrictions[def.position()] = updateRestriction(cfm, def, stmt.columnRestrictions[def.position()], rel, names);
-                         break;
-                     case COMPACT_VALUE:
-                         throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", def.name));
-                     case REGULAR:
-                     case STATIC:
-                         // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
-                         Restriction r = updateRestriction(cfm, def, stmt.metadataRestrictions.get(def.name), rel, names);
-                         if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
-                             // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that slide.
-                             throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", def.name));
-                         stmt.metadataRestrictions.put(def.name, r);
-                         break;
 -                    Term t = relation.getValue().prepare(names);
++                    Term t = relation.getValue().prepare(keyspace(), defs);
+                     t.collectMarkerSpecification(boundNames);
 -                    Restriction restriction = new MultiColumnRestriction.EQ(t, onToken);
 -                    for (CFDefinition.Name name : restrictedColumns)
 -                        stmt.columnRestrictions[name.position] = restriction;
++                    Restriction restriction = new MultiColumnRestriction.EQ(t, false);
++                    for (ColumnDefinition def : restrictedColumns)
++                        stmt.columnRestrictions[def.position()] = restriction;
+                     break;
+                 }
+                 case IN:
+                 {
+                     Restriction restriction;
+                     List<? extends Term.MultiColumnRaw> inValues = relation.getInValues();
+                     if (inValues != null)
+                     {
+                         // we have something like "(a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) or
+                         // "(a, b, c) IN (?, ?, ?)
+                         List<Term> terms = new ArrayList<>(inValues.size());
+                         for (Term.MultiColumnRaw tuple : inValues)
+                         {
 -                            Term t = tuple.prepare(names);
++                            Term t = tuple.prepare(keyspace(), defs);
+                             t.collectMarkerSpecification(boundNames);
+                             terms.add(t);
+                         }
+                          restriction = new MultiColumnRestriction.InWithValues(terms);
+                     }
+                     else
+                     {
+                         Tuples.INRaw rawMarker = relation.getInMarker();
 -                        AbstractMarker t = rawMarker.prepare(names);
++                        AbstractMarker t = rawMarker.prepare(keyspace(), defs);
+                         t.collectMarkerSpecification(boundNames);
+                         restriction = new MultiColumnRestriction.InWithMarker(t);
+                     }
 -                    for (CFDefinition.Name name : restrictedColumns)
 -                        stmt.columnRestrictions[name.position] = restriction;
++                    for (ColumnDefinition def : restrictedColumns)
++                        stmt.columnRestrictions[def.position()] = restriction;
+ 
+                     break;
+                 }
+                 case LT:
+                 case LTE:
+                 case GT:
+                 case GTE:
+                 {
 -                    Term t = relation.getValue().prepare(names);
++                    Term t = relation.getValue().prepare(keyspace(), defs);
+                     t.collectMarkerSpecification(boundNames);
 -                    for (CFDefinition.Name name : names)
++                    for (ColumnDefinition def : defs)
+                     {
 -                        Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, name);
++                        Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, def);
+                         if (restriction == null)
 -                            restriction = new MultiColumnRestriction.Slice(onToken);
++                            restriction = new MultiColumnRestriction.Slice(false);
+                         else if (!restriction.isMultiColumn())
 -                            throw new InvalidRequestException(String.format("Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities", name, relation));
 -                        restriction.setBound(relation.operator(), t);
 -                        stmt.columnRestrictions[name.position] = restriction;
++                            throw new InvalidRequestException(String.format("Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities: %s", def.name, relation));
++                        restriction.setBound(def.name, relation.operator(), t);
++                        stmt.columnRestrictions[def.position()] = restriction;
+                     }
                  }
              }
+         }
  
-             /*
-              * At this point, the select statement if fully constructed, but we still have a few things to validate
-              */
 -        private Restriction getExistingRestriction(SelectStatement stmt, CFDefinition.Name name)
++        private Restriction getExistingRestriction(SelectStatement stmt, ColumnDefinition def)
+         {
 -            switch (name.kind)
++            switch (def.kind)
+             {
 -                case KEY_ALIAS:
 -                    return stmt.keyRestrictions[name.position];
 -                case COLUMN_ALIAS:
 -                    return stmt.columnRestrictions[name.position];
 -                case VALUE_ALIAS:
 -                    return null;
++                case PARTITION_KEY:
++                    return stmt.keyRestrictions[def.position()];
++                case CLUSTERING_COLUMN:
++                    return stmt.columnRestrictions[def.position()];
++                case REGULAR:
++                case STATIC:
++                    return stmt.metadataRestrictions.get(def);
+                 default:
 -                    return stmt.metadataRestrictions.get(name);
++                    throw new AssertionError();
+             }
+         }
  
 -        private void updateRestrictionsForRelation(SelectStatement stmt, CFDefinition.Name name, SingleColumnRelation relation, VariableSpecifications names) throws InvalidRequestException
++        private void updateRestrictionsForRelation(SelectStatement stmt, ColumnDefinition def, SingleColumnRelation relation, VariableSpecifications names) throws InvalidRequestException
+         {
 -            switch (name.kind)
++            switch (def.kind)
+             {
 -                case KEY_ALIAS:
 -                    stmt.keyRestrictions[name.position] = updateSingleColumnRestriction(name, stmt.keyRestrictions[name.position], relation, names);
++                case PARTITION_KEY:
++                    stmt.keyRestrictions[def.position()] = updateSingleColumnRestriction(def, stmt.keyRestrictions[def.position()], relation, names);
+                     break;
 -                case COLUMN_ALIAS:
 -                    stmt.columnRestrictions[name.position] = updateSingleColumnRestriction(name, stmt.columnRestrictions[name.position], relation, names);
++                case CLUSTERING_COLUMN:
++                    stmt.columnRestrictions[def.position()] = updateSingleColumnRestriction(def, stmt.columnRestrictions[def.position()], relation, names);
+                     break;
 -                case VALUE_ALIAS:
 -                    throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", name.name));
 -                case COLUMN_METADATA:
++                case COMPACT_VALUE:
++                    throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", def.name));
++                case REGULAR:
+                 case STATIC:
+                     // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
 -                    Restriction r = updateSingleColumnRestriction(name, stmt.metadataRestrictions.get(name), relation, names);
++                    Restriction r = updateSingleColumnRestriction(def, stmt.metadataRestrictions.get(def.name), relation, names);
+                     if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
+                         // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that slide.
 -                        throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", name));
 -                    stmt.metadataRestrictions.put(name, r);
++                        throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", def.name));
++                    stmt.metadataRestrictions.put(def.name, r);
+                     break;
+             }
+         }
+ 
 -        Restriction updateSingleColumnRestriction(CFDefinition.Name name, Restriction existingRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
++        Restriction updateSingleColumnRestriction(ColumnDefinition def, Restriction existingRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
+         {
 -            ColumnSpecification receiver = name;
++            ColumnSpecification receiver = def;
+             if (newRel.onToken)
+             {
 -                if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS)
 -                    throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", name));
++                if (def.kind != ColumnDefinition.Kind.PARTITION_KEY)
++                    throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", def.name));
+ 
 -                receiver = new ColumnSpecification(name.ksName,
 -                                                   name.cfName,
++                receiver = new ColumnSpecification(def.ksName,
++                                                   def.cfName,
+                                                    new ColumnIdentifier("partition key token", true),
+                                                    StorageService.getPartitioner().getTokenValidator());
+             }
+ 
+             switch (newRel.operator())
+             {
+                 case EQ:
+                 {
+                     if (existingRestriction != null)
 -                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", name));
 -                    Term t = newRel.getValue().prepare(receiver);
++                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", def.name));
++                    Term t = newRel.getValue().prepare(keyspace(), receiver);
+                     t.collectMarkerSpecification(boundNames);
+                     existingRestriction = new SingleColumnRestriction.EQ(t, newRel.onToken);
+                 }
+                 break;
+                 case IN:
+                     if (existingRestriction != null)
 -                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", name));
++                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", def.name));
+ 
+                     if (newRel.getInValues() == null)
+                     {
+                         // Means we have a "SELECT ... IN ?"
+                         assert newRel.getValue() != null;
 -                        Term t = newRel.getValue().prepare(receiver);
++                        Term t = newRel.getValue().prepare(keyspace(), receiver);
+                         t.collectMarkerSpecification(boundNames);
+                         existingRestriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
+                     }
+                     else
+                     {
 -                        List<Term> inValues = new ArrayList<Term>(newRel.getInValues().size());
++                        List<Term> inValues = new ArrayList<>(newRel.getInValues().size());
+                         for (Term.Raw raw : newRel.getInValues())
+                         {
 -                            Term t = raw.prepare(receiver);
++                            Term t = raw.prepare(keyspace(), receiver);
+                             t.collectMarkerSpecification(boundNames);
+                             inValues.add(t);
+                         }
+                         existingRestriction = new SingleColumnRestriction.InWithValues(inValues);
+                     }
+                     break;
+                 case GT:
+                 case GTE:
+                 case LT:
+                 case LTE:
++                    {
++                        if (existingRestriction == null)
++                            existingRestriction = new SingleColumnRestriction.Slice(newRel.onToken);
++                        else if (!existingRestriction.isSlice())
++                            throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both an equality and an inequality relation", def.name));
++                        else if (existingRestriction.isMultiColumn())
++                            throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both a tuple notation inequality and a single column inequality (%s)", def.name, newRel));
++
++                        Term t = newRel.getValue().prepare(keyspace(), receiver);
++                        t.collectMarkerSpecification(boundNames);
++                        ((SingleColumnRestriction.Slice)existingRestriction).setBound(def.name, newRel.operator(), t);
++                    }
++                    break;
++                case CONTAINS_KEY:
++                    if (!(receiver.type instanceof MapType))
++                        throw new InvalidRequestException(String.format("Cannot use CONTAINS_KEY on non-map column %s", def.name));
++                    // Fallthrough on purpose
++                case CONTAINS:
+                 {
++                    if (!receiver.type.isCollection())
++                        throw new InvalidRequestException(String.format("Cannot use %s relation on non collection column %s", newRel.operator(), def.name));
++
+                     if (existingRestriction == null)
 -                        existingRestriction = new SingleColumnRestriction.Slice(newRel.onToken);
 -                    else if (!existingRestriction.isSlice())
 -                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both an equality and an inequality relation", name));
 -                    else if (existingRestriction.isMultiColumn())
 -                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both a tuple notation inequality and a single column inequality (%s)", name, newRel));
 -                    Term t = newRel.getValue().prepare(receiver);
 -                    t.collectMarkerSpecification(boundNames);
 -                    ((SingleColumnRestriction.Slice)existingRestriction).setBound(newRel.operator(), t);
++                        existingRestriction = new SingleColumnRestriction.Contains();
++                    else if (!existingRestriction.isContains())
++                        throw new InvalidRequestException(String.format("Collection column %s can only be restricted by CONTAINS or CONTAINS KEY", def.name));
++                    boolean isKey = newRel.operator() == Relation.Type.CONTAINS_KEY;
++                    receiver = makeCollectionReceiver(receiver, isKey);
++                    Term t = newRel.getValue().prepare(keyspace(), receiver);
++                    ((SingleColumnRestriction.Contains)existingRestriction).add(t, isKey);
+                 }
 -                break;
+             }
+             return existingRestriction;
+         }
+ 
 -        private void processPartitionKeyRestrictions(SelectStatement stmt, CFDefinition cfDef, boolean hasQueriableIndex) throws InvalidRequestException
++        private void processPartitionKeyRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException
+         {
              // If there is a queriable index, no special condition are required on the other restrictions.
              // But we still need to know 2 things:
              //   - If we don't have a queriable index, is the query ok
@@@ -1338,7 -1734,9 +1712,9 @@@
                          stmt.usesSecondaryIndexing = true;
                          break;
                      }
-                     throw new InvalidRequestException(String.format("partition key part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cdef.name, previous));
+                     throw new InvalidRequestException(String.format(
+                             "Partitioning column \"%s\" cannot be restricted because the preceding column (\"%s\") is " +
 -                            "either not restricted or is restricted by a non-EQ relation", cname, previous));
++                            "either not restricted or is restricted by a non-EQ relation", cdef.name, previous));
                  }
                  else if (restriction.isOnToken())
                  {
@@@ -1368,27 -1766,22 +1744,22 @@@
                      // index with filtering, we'll need to handle it though.
                      throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
                  }
 -                previous = cname;
 +                previous = cdef;
              }
+         }
  
-             // All (or none) of the partition key columns have been specified;
-             // hence there is no need to turn these restrictions into index expressions.
-             if (!stmt.usesSecondaryIndexing)
-                 stmt.restrictedColumns.removeAll(cfm.partitionKeyColumns());
- 
-             if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
-                 throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
- 
 -        private void processColumnRestrictions(SelectStatement stmt, CFDefinition cfDef, boolean hasQueriableIndex) throws InvalidRequestException
++        private void processColumnRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException
+         {
              // If a clustering key column is restricted by a non-EQ relation, all preceding
              // columns must have a EQ, and all following must have no restriction. Unless
              // the column is indexed that is.
-             canRestrictFurtherComponents = true;
-             previous = null;
+             boolean canRestrictFurtherComponents = true;
 -            CFDefinition.Name previous = null;
++            ColumnDefinition previous = null;
              boolean previousIsSlice = false;
-             iter = cfm.clusteringColumns().iterator();
 -            Iterator<CFDefinition.Name> iter = cfDef.clusteringColumns().iterator();
++            Iterator<ColumnDefinition> iter = cfm.clusteringColumns().iterator();
              for (int i = 0; i < stmt.columnRestrictions.length; i++)
              {
 -                CFDefinition.Name cname = iter.next();
 +                ColumnDefinition cdef = iter.next();
                  Restriction restriction = stmt.columnRestrictions[i];
  
                  if (restriction == null)
@@@ -1410,7 -1803,8 +1781,8 @@@
                              stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
                              break;
                          }
-                         throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cdef.name, previous));
+                         throw new InvalidRequestException(String.format(
 -                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is either not restricted or by a non-EQ relation)", cname, previous));
++                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is either not restricted or by a non-EQ relation)", cdef.name, previous));
                      }
                  }
                  else if (restriction.isSlice())
@@@ -1425,160 -1819,160 +1797,152 @@@
                  }
                  else if (restriction.isIN())
                  {
-                     // We only support IN for the last name and for compact storage so far
-                     // TODO: #3885 allows us to extend to non compact as well, but that remains to be done
-                     if (i != stmt.columnRestrictions.length - 1)
-                         throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cdef.name));
+                     if (!restriction.isMultiColumn() && i != stmt.columnRestrictions.length - 1)
 -                        throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cname));
 -                    if (stmt.selectACollection())
 -                        throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cname));
++                        throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cdef.name));
 +                    else if (stmt.selectACollection())
-                         throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cdef.name));
++                        throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cdef.name));
                  }
  
 -                previous = cname;
 +                previous = cdef;
              }
+         }
  
-             // Covers indexes on the first clustering column (among others).
-             if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
-                 stmt.usesSecondaryIndexing = true;
- 
-             if (!stmt.usesSecondaryIndexing)
-                 stmt.restrictedColumns.removeAll(cfm.clusteringColumns());
- 
-             // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
-             // there is restrictions not covered by the PK.
-             if (!stmt.metadataRestrictions.isEmpty())
-             {
-                 if (!hasQueriableIndex)
-                     throw new InvalidRequestException("No indexed columns present in by-columns clause with Equal operator");
-                 stmt.usesSecondaryIndexing = true;
-             }
+         private void validateSecondaryIndexSelections(SelectStatement stmt) throws InvalidRequestException
+         {
+             if (stmt.keyIsInRelation)
+                 throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
+             // When the user only select static columns, the intent is that we don't query the whole partition but just
+             // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
+             // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
+             if (stmt.selectsOnlyStaticColumns)
+                 throw new InvalidRequestException("Queries using 2ndary indexes don't support selecting only static columns");
+         }
  
+         private void verifyOrderingIsAllowed(SelectStatement stmt) throws InvalidRequestException
+         {
              if (stmt.usesSecondaryIndexing)
-             {
-                 if (stmt.keyIsInRelation)
-                     throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
-                 // When the user only select static columns, the intent is that we don't query the whole partition but just
-                 // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
-                 // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
-                 if (stmt.selectsOnlyStaticColumns)
-                     throw new InvalidRequestException("Queries using 2ndary indexes don't support selecting only static columns");
-             }
+                 throw new InvalidRequestException("ORDER BY with 2ndary indexes is not supported.");
  
-             if (!stmt.parameters.orderings.isEmpty())
-             {
-                 if (stmt.usesSecondaryIndexing)
-                     throw new InvalidRequestException("ORDER BY with 2ndary indexes is not supported.");
+             if (stmt.isKeyRange)
+                 throw new InvalidRequestException("ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
+         }
  
-                 if (stmt.isKeyRange)
-                     throw new InvalidRequestException("ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
+         private void handleUnrecognizedOrderingColumn(ColumnIdentifier column) throws InvalidRequestException
+         {
+             if (containsAlias(column))
+                 throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
+             else
+                 throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
+         }
  
-                 // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting, even if we don't
-                 // ultimately ship them to the client (CASSANDRA-4911).
-                 if (stmt.keyIsInRelation)
 -        private void processOrderingClause(SelectStatement stmt, CFDefinition cfDef) throws InvalidRequestException
++        private void processOrderingClause(SelectStatement stmt, CFMetaData cfm) throws InvalidRequestException
+         {
+             verifyOrderingIsAllowed(stmt);
+ 
 -            // If we order an IN query, we'll have to do a manual sort post-query. Currently, this sorting requires that we
 -            // have queried the column on which we sort (TODO: we should update it to add the column on which we sort to the one
 -            // queried automatically, and then removing it from the resultSet afterwards if needed)
++            // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting, even if we don't
++            // ultimately ship them to the client (CASSANDRA-4911).
+             if (stmt.keyIsInRelation)
+             {
 -                stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
++                stmt.orderingIndexes = new HashMap<>();
+                 for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
                  {
-                     stmt.orderingIndexes = new HashMap<>();
-                     for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
-                     {
-                         final ColumnDefinition def = cfm.getColumnDefinition(column);
-                         if (def == null)
-                         {
-                             if (containsAlias(column))
-                                 throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
-                             else
-                                 throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
-                         }
 -                    final CFDefinition.Name name = cfDef.get(column);
 -                    if (name == null)
++                    final ColumnDefinition def = cfm.getColumnDefinition(column);
++                    if (def == null)
+                         handleUnrecognizedOrderingColumn(column);
  
-                         int index = indexOf(def, stmt.selection);
-                         if (index < 0)
-                             index = stmt.selection.addColumnForOrdering(def);
-                         stmt.orderingIndexes.put(def.name, index);
-                     }
 -                    if (selectClause.isEmpty()) // wildcard
 -                    {
 -                        stmt.orderingIndexes.put(name, Iterables.indexOf(cfDef, new Predicate<CFDefinition.Name>()
 -                        {
 -                            public boolean apply(CFDefinition.Name n)
 -                            {
 -                                return name.equals(n);
 -                            }
 -                        }));
 -                    }
 -                    else
 -                    {
 -                        boolean hasColumn = false;
 -                        for (int i = 0; i < selectClause.size(); i++)
 -                        {
 -                            RawSelector selector = selectClause.get(i);
 -                            if (name.name.equals(selector.selectable))
 -                            {
 -                                stmt.orderingIndexes.put(name, i);
 -                                hasColumn = true;
 -                                break;
 -                            }
 -                        }
 -
 -                        if (!hasColumn)
 -                            throw new InvalidRequestException("ORDER BY could not be used on columns missing in select clause.");
 -                    }
++                    int index = indexOf(def, stmt.selection);
++                    if (index < 0)
++                        index = stmt.selection.addColumnForOrdering(def);
++                    stmt.orderingIndexes.put(def.name, index);
                  }
+             }
 -            stmt.isReversed = isReversed(stmt, cfDef);
++            stmt.isReversed = isReversed(stmt, cfm);
+         }
  
-                 Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
-                 int i = 0;
-                 for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
-                 {
-                     ColumnIdentifier column = entry.getKey();
-                     boolean reversed = entry.getValue();
 -        private boolean isReversed(SelectStatement stmt, CFDefinition cfDef) throws InvalidRequestException
++        private boolean isReversed(SelectStatement stmt, CFMetaData cfm) throws InvalidRequestException
+         {
 -            Boolean[] reversedMap = new Boolean[cfDef.clusteringColumnsCount()];
++            Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
+             int i = 0;
+             for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
+             {
+                 ColumnIdentifier column = entry.getKey();
+                 boolean reversed = entry.getValue();
  
-                     ColumnDefinition def = cfm.getColumnDefinition(column);
-                     if (def == null)
-                     {
-                         if (containsAlias(column))
-                             throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
-                         else
-                             throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
-                     }
 -                CFDefinition.Name name = cfDef.get(column);
 -                if (name == null)
++                ColumnDefinition def = cfm.getColumnDefinition(column);
++                if (def == null)
+                     handleUnrecognizedOrderingColumn(column);
  
-                     if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
-                         throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
 -                if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
++                if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
+                     throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
  
-                     if (i++ != def.position())
-                         throw new InvalidRequestException(String.format("Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"));
 -                if (i++ != name.position)
++                if (i++ != def.position())
+                     throw new InvalidRequestException(String.format("Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"));
  
-                     reversedMap[def.position()] = (reversed != isReversedType(def));
-                 }
 -                reversedMap[name.position] = (reversed != isReversedType(name));
++                reversedMap[def.position()] = (reversed != isReversedType(def));
+             }
  
-                 // Check that all boolean in reversedMap, if set, agrees
-                 Boolean isReversed = null;
-                 for (Boolean b : reversedMap)
-                 {
-                     // Cell on which order is specified can be in any order
-                     if (b == null)
-                         continue;
+             // Check that all boolean in reversedMap, if set, agrees
+             Boolean isReversed = null;
+             for (Boolean b : reversedMap)
+             {
+                 // Column on which order is specified can be in any order
+                 if (b == null)
+                     continue;
  
-                     if (isReversed == null)
-                     {
-                         isReversed = b;
-                         continue;
-                     }
-                     if (isReversed != b)
-                         throw new InvalidRequestException(String.format("Unsupported order by relation"));
+                 if (isReversed == null)
+                 {
+                     isReversed = b;
+                     continue;
                  }
-                 assert isReversed != null;
-                 stmt.isReversed = isReversed;
+                 if (isReversed != b)
+                     throw new InvalidRequestException(String.format("Unsupported order by relation"));
              }
+             assert isReversed != null;
+             return isReversed;
+         }
  
-             // Make sure this queries is allowed (note: non key range non indexed cannot involve filtering underneath)
+         /** If ALLOW FILTERING was not specified, this verifies that it is not needed */
+         private void checkNeedsFiltering(SelectStatement stmt) throws InvalidRequestException
+         {
+             // non-key-range non-indexed queries cannot involve filtering underneath
              if (!parameters.allowFiltering && (stmt.isKeyRange || stmt.usesSecondaryIndexing))
              {
                  // We will potentially filter data if either:
                  //  - Have more than one IndexExpression
                  //  - Have no index expression and the column filter is not the identity
 -                if (stmt.restrictedNames.size() > 1 || (stmt.restrictedNames.isEmpty() && !stmt.columnFilterIsIdentity()))
 +                if (stmt.restrictedColumns.size() > 1 || (stmt.restrictedColumns.isEmpty() && !stmt.columnFilterIsIdentity()))
-                     throw new InvalidRequestException("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. "
-                                                     + "If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING");
+                     throw new InvalidRequestException("Cannot execute this query as it might involve data filtering and " +
+                                                       "thus may have unpredictable performance. If you want to execute " +
+                                                       "this query despite the performance unpredictability, use ALLOW FILTERING");
              }
- 
-             return new ParsedStatement.Prepared(stmt, names);
          }
  
 -        private void validateDistinctSelection(Collection<CFDefinition.Name> requestedColumns, Collection<CFDefinition.Name> partitionKey)
 +        private int indexOf(ColumnDefinition def, Selection selection)
 +        {
 +            return indexOf(def, selection.getColumns().iterator());
 +        }
 +
 +        private int indexOf(final ColumnDefinition def, Iterator<ColumnDefinition> defs)
 +        {
 +            return Iterators.indexOf(defs, new Predicate<ColumnDefinition>()
 +                                           {
 +                                               public boolean apply(ColumnDefinition n)
 +                                               {
 +                                                   return def.name.equals(n.name);
 +                                               }
 +                                           });
 +        }
 +
 +        private void validateDistinctSelection(Collection<ColumnDefinition> requestedColumns, Collection<ColumnDefinition> partitionKey)
          throws InvalidRequestException
          {
 -            for (CFDefinition.Name name : requestedColumns)
 -                if (!partitionKey.contains(name))
 -                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", name));
 +            for (ColumnDefinition def : requestedColumns)
 +                if (!partitionKey.contains(def))
 +                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", def.name));
  
 -            for (CFDefinition.Name name : partitionKey)
 -                if (!requestedColumns.contains(name))
 -                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
 +            for (ColumnDefinition def : partitionKey)
 +                if (!requestedColumns.contains(def))
 +                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name));
          }
  
          private boolean containsAlias(final ColumnIdentifier name)
@@@ -1597,114 -1991,6 +1961,23 @@@
              return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
          }
  
-         Restriction updateRestriction(CFMetaData cfm, ColumnDefinition def, Restriction restriction, Relation newRel, VariableSpecifications boundNames) throws InvalidRequestException
-         {
-             ColumnSpecification receiver = def;
-             if (newRel.onToken)
-             {
-                 if (def.kind != ColumnDefinition.Kind.PARTITION_KEY)
-                     throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", def.name));
- 
-                 receiver = new ColumnSpecification(def.ksName,
-                                                    def.cfName,
-                                                    new ColumnIdentifier("partition key token", true),
-                                                    StorageService.getPartitioner().getTokenValidator());
-             }
- 
-             // We can only use the tuple notation of #4851 on clustering columns for now
-             if (newRel.previousInTuple != null && def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
-                 throw new InvalidRequestException(String.format("Tuple notation can only be used on clustering columns but found on %s", def.name));
- 
-             switch (newRel.operator())
-             {
-                 case EQ:
-                     {
-                         if (restriction != null)
-                             throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", def.name));
-                         Term t = newRel.getValue().prepare(keyspace(), receiver);
-                         t.collectMarkerSpecification(boundNames);
-                         restriction = new Restriction.EQ(t, newRel.onToken);
-                     }
-                     break;
-                 case IN:
-                     if (restriction != null)
-                         throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", def.name));
- 
-                     if (newRel.getInValues() == null)
-                     {
-                         // Means we have a "SELECT ... IN ?"
-                         assert newRel.getValue() != null;
-                         Term t = newRel.getValue().prepare(keyspace(), receiver);
-                         t.collectMarkerSpecification(boundNames);
-                         restriction = Restriction.IN.create(t);
-                     }
-                     else
-                     {
-                         List<Term> inValues = new ArrayList<Term>(newRel.getInValues().size());
-                         for (Term.Raw raw : newRel.getInValues())
-                         {
-                             Term t = raw.prepare(keyspace(), receiver);
-                             t.collectMarkerSpecification(boundNames);
-                             inValues.add(t);
-                         }
-                         restriction = Restriction.IN.create(inValues);
-                     }
-                     break;
-                 case GT:
-                 case GTE:
-                 case LT:
-                 case LTE:
-                     {
-                         if (restriction == null)
-                             restriction = new Restriction.Slice(newRel.onToken);
-                         else if (!restriction.isSlice())
-                             throw new InvalidRequestException(String.format("%s cannot be restricted by both an equal and an inequal relation", def.name));
-                         Term t = newRel.getValue().prepare(keyspace(), receiver);
-                         t.collectMarkerSpecification(boundNames);
-                         if (newRel.previousInTuple != null && (def.position() == 0 || !cfm.clusteringColumns().get(def.position() - 1).name.equals(newRel.previousInTuple)))
-                             throw new InvalidRequestException(String.format("Invalid tuple notation, column %s is not before column %s in the clustering order", newRel.previousInTuple, def.name));
-                         ((Restriction.Slice)restriction).setBound(def.name, newRel.operator(), t, newRel.previousInTuple);
-                     }
-                     break;
-                 case CONTAINS_KEY:
-                     if (!(receiver.type instanceof MapType))
-                         throw new InvalidRequestException(String.format("Cannot use CONTAINS_KEY on non-map column %s", def.name));
-                     // Fallthrough on purpose
-                 case CONTAINS:
-                     {
-                         if (!receiver.type.isCollection())
-                             throw new InvalidRequestException(String.format("Cannot use %s relation on non collection column %s", newRel.operator(), def.name));
- 
-                         if (restriction == null)
-                             restriction = new Restriction.Contains();
-                         else if (!restriction.isContains())
-                             throw new InvalidRequestException(String.format("Collection column %s can only be restricted by CONTAINS or CONTAINS KEY", def.name));
-                         boolean isKey = newRel.operator() == Relation.Type.CONTAINS_KEY;
-                         receiver = makeCollectionReceiver(receiver, isKey);
-                         Term t = newRel.getValue().prepare(keyspace(), receiver);
-                         ((Restriction.Contains)restriction).add(t, isKey);
-                     }
-             }
-             return restriction;
-         }
- 
 +        private static ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
 +        {
 +            assert collection.type.isCollection();
 +            switch (((CollectionType)collection.type).kind)
 +            {
 +                case LIST:
 +                    assert !isKey;
 +                    return Lists.valueSpecOf(collection);
 +                case SET:
 +                    assert !isKey;
 +                    return Sets.valueSpecOf(collection);
 +                case MAP:
 +                    return isKey ? Maps.keySpecOf(collection) : Maps.valueSpecOf(collection);
 +            }
 +            throw new AssertionError();
 +        }
 +
          @Override
          public String toString()
          {


[4/7] Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
index 0000000,2e63272..bc77357
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
@@@ -1,0 -1,300 +1,413 @@@
+ /*
+  * 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.statements;
+ 
+ import org.apache.cassandra.cql3.*;
++import org.apache.cassandra.db.IndexExpression;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
 -import org.apache.cassandra.thrift.IndexOperator;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.ArrayList;
+ import java.util.Collections;
+ import java.util.List;
+ 
+ public abstract class SingleColumnRestriction implements Restriction
+ {
+     public boolean isMultiColumn()
+     {
+         return false;
+     }
+ 
+     public static class EQ extends SingleColumnRestriction implements Restriction.EQ
+     {
+         protected final Term value;
+         private final boolean onToken;
+ 
+         public EQ(Term value, boolean onToken)
+         {
+             this.value = value;
+             this.onToken = onToken;
+         }
+ 
 -        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
++        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+         {
 -            return Collections.singletonList(value.bindAndGet(variables));
++            return Collections.singletonList(value.bindAndGet(options));
+         }
+ 
+         public boolean isSlice()
+         {
+             return false;
+         }
+ 
+         public boolean isEQ()
+         {
+             return true;
+         }
+ 
+         public boolean isIN()
+         {
+             return false;
+         }
+ 
++        public boolean isContains()
++        {
++            return false;
++        }
++
+         public boolean isOnToken()
+         {
+             return onToken;
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return String.format("EQ(%s)%s", value, onToken ? "*" : "");
+         }
+     }
+ 
+     public static class InWithValues extends SingleColumnRestriction implements Restriction.IN
+     {
+         protected final List<Term> values;
+ 
+         public InWithValues(List<Term> values)
+         {
+             this.values = values;
+         }
+ 
 -        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
++        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+         {
+             List<ByteBuffer> buffers = new ArrayList<>(values.size());
+             for (Term value : values)
 -                buffers.add(value.bindAndGet(variables));
++                buffers.add(value.bindAndGet(options));
+             return buffers;
+         }
+ 
+         public boolean canHaveOnlyOneValue()
+         {
+             return values.size() == 1;
+         }
+ 
+         public boolean isSlice()
+         {
+             return false;
+         }
+ 
+         public boolean isEQ()
+         {
+             return false;
+         }
+ 
+         public boolean isIN()
+         {
+             return true;
+         }
+ 
++        public boolean isContains()
++        {
++            return false;
++        }
++
+         public boolean isOnToken()
+         {
+             return false;
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return String.format("IN(%s)", values);
+         }
+     }
+ 
+     public static class InWithMarker extends SingleColumnRestriction implements Restriction.IN
+     {
+         protected final AbstractMarker marker;
+ 
+         public InWithMarker(AbstractMarker marker)
+         {
+             this.marker = marker;
+         }
+ 
 -        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
++        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+         {
 -            Term.MultiItemTerminal lval = (Term.MultiItemTerminal)marker.bind(variables);
++            Term.MultiItemTerminal lval = (Term.MultiItemTerminal)marker.bind(options);
+             if (lval == null)
+                 throw new InvalidRequestException("Invalid null value for IN restriction");
+             return lval.getElements();
+         }
+ 
+         public boolean canHaveOnlyOneValue()
+         {
+             return false;
+         }
+ 
+         public boolean isSlice()
+         {
+             return false;
+         }
+ 
+         public boolean isEQ()
+         {
+             return false;
+         }
+ 
+         public boolean isIN()
+         {
+             return true;
+         }
+ 
++        public boolean isContains()
++        {
++            return false;
++        }
++
+         public boolean isOnToken()
+         {
+             return false;
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return "IN ?";
+         }
+     }
+ 
+     public static class Slice extends SingleColumnRestriction implements Restriction.Slice
+     {
+         protected final Term[] bounds;
+         protected final boolean[] boundInclusive;
+         protected final boolean onToken;
+ 
+         public Slice(boolean onToken)
+         {
+             this.bounds = new Term[2];
+             this.boundInclusive = new boolean[2];
+             this.onToken = onToken;
+         }
+ 
+         public boolean isSlice()
+         {
+             return true;
+         }
+ 
+         public boolean isEQ()
+         {
+             return false;
+         }
+ 
+         public boolean isIN()
+         {
+             return false;
+         }
+ 
 -        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
++        public boolean isContains()
++        {
++            return false;
++        }
++
++        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+         {
+             throw new UnsupportedOperationException();
+         }
+ 
+         public boolean isOnToken()
+         {
+             return onToken;
+         }
+ 
+         /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
+         public boolean hasBound(Bound b)
+         {
+             return bounds[b.idx] != null;
+         }
+ 
 -        public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
++        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
+         {
 -            return bounds[b.idx].bindAndGet(variables);
++            return bounds[b.idx].bindAndGet(options);
+         }
+ 
+         /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
+         public boolean isInclusive(Bound b)
+         {
+             return bounds[b.idx] == null || boundInclusive[b.idx];
+         }
+ 
+         public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound)
+         {
+             switch (eocBound)
+             {
+                 case START:
+                     return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT;
+                 case END:
+                     return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT;
+             }
+             throw new AssertionError();
+         }
+ 
 -        public IndexOperator getIndexOperator(Bound b)
++        public IndexExpression.Operator getIndexOperator(Bound b)
+         {
+             switch (b)
+             {
+                 case START:
 -                    return boundInclusive[b.idx] ? IndexOperator.GTE : IndexOperator.GT;
++                    return boundInclusive[b.idx] ? IndexExpression.Operator.GTE : IndexExpression.Operator.GT;
+                 case END:
 -                    return boundInclusive[b.idx] ? IndexOperator.LTE : IndexOperator.LT;
++                    return boundInclusive[b.idx] ? IndexExpression.Operator.LTE : IndexExpression.Operator.LT;
+             }
+             throw new AssertionError();
+         }
+ 
 -        public void setBound(Relation.Type type, Term t) throws InvalidRequestException
++        public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException
+         {
+             Bound b;
+             boolean inclusive;
+             switch (type)
+             {
+                 case GT:
+                     b = Bound.START;
+                     inclusive = false;
+                     break;
+                 case GTE:
+                     b = Bound.START;
+                     inclusive = true;
+                     break;
+                 case LT:
+                     b = Bound.END;
+                     inclusive = false;
+                     break;
+                 case LTE:
+                     b = Bound.END;
+                     inclusive = true;
+                     break;
+                 default:
+                     throw new AssertionError();
+             }
+ 
+             if (bounds[b.idx] != null)
+                 throw new InvalidRequestException(String.format(
 -                        "More than one restriction was found for the %s bound", b.name().toLowerCase()));
++                        "More than one restriction was found for the %s bound on %s", b.name().toLowerCase(), name));
+ 
+             bounds[b.idx] = t;
+             boundInclusive[b.idx] = inclusive;
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return String.format("SLICE(%s %s, %s %s)%s", boundInclusive[0] ? ">=" : ">",
+                                  bounds[0],
+                                  boundInclusive[1] ? "<=" : "<",
+                                  bounds[1],
+                                  onToken ? "*" : "");
+         }
+     }
++
++    // This holds both CONTAINS and CONTAINS_KEY restriction because we might want to have both of them.
++    public static class Contains extends SingleColumnRestriction
++    {
++        private List<Term> values; // for CONTAINS
++        private List<Term> keys;   // for CONTAINS_KEY
++
++        public boolean hasContains()
++        {
++            return values != null;
++        }
++
++        public boolean hasContainsKey()
++        {
++            return keys != null;
++        }
++
++        public void add(Term t, boolean isKey)
++        {
++            if (isKey)
++                addKey(t);
++            else
++                addValue(t);
++        }
++
++        public void addValue(Term t)
++        {
++            if (values == null)
++                values = new ArrayList<>();
++            values.add(t);
++        }
++
++        public void addKey(Term t)
++        {
++            if (keys == null)
++                keys = new ArrayList<>();
++            keys.add(t);
++        }
++
++        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++        {
++            if (values == null)
++                return Collections.emptyList();
++
++            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(values.size());
++            for (Term value : values)
++                buffers.add(value.bindAndGet(options));
++            return buffers;
++        }
++
++        public List<ByteBuffer> keys(QueryOptions options) throws InvalidRequestException
++        {
++            if (keys == null)
++                return Collections.emptyList();
++
++            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(keys.size());
++            for (Term value : keys)
++                buffers.add(value.bindAndGet(options));
++            return buffers;
++        }
++
++        public boolean isSlice()
++        {
++            return false;
++        }
++
++        public boolean isEQ()
++        {
++            return false;
++        }
++
++        public boolean isIN()
++        {
++            return false;
++        }
++
++        public boolean isContains()
++        {
++            return true;
++        }
++
++        public boolean isOnToken()
++        {
++            return false;
++        }
++
++
++        @Override
++        public String toString()
++        {
++            return String.format("CONTAINS(values=%s, keys=%s)", values, keys);
++        }
++    }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/db/composites/CBuilder.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CBuilder.java
index 490fb67,0000000..39035cb
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CBuilder.java
+++ b/src/java/org/apache/cassandra/db/composites/CBuilder.java
@@@ -1,34 -1,0 +1,36 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
++import java.util.List;
 +
 +/**
 + * A builder of Composite.
 + */
 +public interface CBuilder
 +{
 +    public int remainingCount();
 +
 +    public CBuilder add(ByteBuffer value);
 +    public CBuilder add(Object value);
 +
 +    public Composite build();
 +    public Composite buildWith(ByteBuffer value);
++    public Composite buildWith(List<ByteBuffer> values);
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/db/composites/Composites.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/Composites.java
index 58938c6,0000000..154e9f7
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/Composites.java
+++ b/src/java/org/apache/cassandra/db/composites/Composites.java
@@@ -1,125 -1,0 +1,127 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
++import java.util.List;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +public abstract class Composites
 +{
 +    private Composites() {}
 +
 +    public static final Composite EMPTY = new EmptyComposite();
 +
 +    static final CBuilder EMPTY_BUILDER = new CBuilder()
 +    {
 +        public int remainingCount() { return 0; }
 +
 +        public CBuilder add(ByteBuffer value) { throw new IllegalStateException(); }
 +        public CBuilder add(Object value) { throw new IllegalStateException(); }
 +
 +        public Composite build() { return EMPTY; }
 +        public Composite buildWith(ByteBuffer value) { throw new IllegalStateException(); }
++        public Composite buildWith(List<ByteBuffer> values) { throw new IllegalStateException(); }
 +    };
 +
 +    private static class EmptyComposite implements Composite
 +    {
 +        public boolean isEmpty()
 +        {
 +            return true;
 +        }
 +
 +        public int size()
 +        {
 +            return 0;
 +        }
 +
 +        public ByteBuffer get(int i)
 +        {
 +            throw new IndexOutOfBoundsException();
 +        }
 +
 +        public EOC eoc()
 +        {
 +            return EOC.NONE;
 +        }
 +
 +        public Composite start()
 +        {
 +            // Note that SimpleCType/AbstractSimpleCellNameType compare method
 +            // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end()
 +            // (or more precisely on the fact that the EOC is NONE for all of those).
 +            return this;
 +        }
 +
 +        public Composite end()
 +        {
 +            // Note that SimpleCType/AbstractSimpleCellNameType compare method
 +            // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end()
 +            // (or more precisely on the fact that the EOC is NONE for all of those).
 +            return this;
 +        }
 +
 +        public Composite withEOC(EOC newEoc)
 +        {
 +            // Note that SimpleCType/AbstractSimpleCellNameType compare method
 +            // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end()
 +            // (or more precisely on the fact that the EOC is NONE for all of those).
 +            return this;
 +        }
 +
 +        public ColumnSlice slice()
 +        {
 +            return ColumnSlice.ALL_COLUMNS;
 +        }
 +
 +        public ByteBuffer toByteBuffer()
 +        {
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +        }
 +
 +        public boolean isStatic()
 +        {
 +            return false;
 +        }
 +
 +        public int dataSize()
 +        {
 +            return 0;
 +        }
 +
 +        public long unsharedHeapSize()
 +        {
 +            return 0;
 +        }
 +
 +        public boolean isPrefixOf(CType type, Composite c)
 +        {
 +            return true;
 +        }
 +
 +        public Composite copy(CFMetaData cfm, AbstractAllocator allocator)
 +        {
 +            return this;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/db/composites/CompoundCType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundCType.java
index 4322055,0000000..0458748
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundCType.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundCType.java
@@@ -1,167 -1,0 +1,180 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.List;
 +
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CompositeType;
 +
 +/**
 + * A truly-composite CType.
 + */
 +public class CompoundCType extends AbstractCType
 +{
 +    final List<AbstractType<?>> types;
 +
 +    // It's up to the caller to pass a list that is effectively immutable
 +    public CompoundCType(List<AbstractType<?>> types)
 +    {
 +        super(isByteOrderComparable(types));
 +        this.types = types;
 +    }
 +
 +    public boolean isCompound()
 +    {
 +        return true;
 +    }
 +
 +    public int size()
 +    {
 +        return types.size();
 +    }
 +
 +    public AbstractType<?> subtype(int i)
 +    {
 +        return types.get(i);
 +    }
 +
 +    public Composite fromByteBuffer(ByteBuffer bytes)
 +    {
 +        if (!bytes.hasRemaining())
 +            return Composites.EMPTY;
 +
 +        ByteBuffer[] elements = new ByteBuffer[size()];
 +        int idx = bytes.position(), i = 0;
 +        byte eoc = 0;
 +
 +        boolean isStatic = false;
 +        if (CompositeType.isStaticName(bytes))
 +        {
 +            isStatic = true;
 +            idx += 2;
 +        }
 +
 +        while (idx < bytes.limit())
 +        {
 +            checkRemaining(bytes, idx, 2);
 +            int length = bytes.getShort(idx) & 0xFFFF;
 +            idx += 2;
 +
 +            checkRemaining(bytes, idx, length + 1);
 +            elements[i++] = sliceBytes(bytes, idx, length);
 +            idx += length;
 +            eoc = bytes.get(idx++);
 +        }
 +        return new CompoundComposite(elements, i, isStatic).withEOC(Composite.EOC.from(eoc));
 +    }
 +
 +    public CBuilder builder()
 +    {
 +        return new CompoundCBuilder(this);
 +    }
 +
 +    public CompoundCType setSubtype(int position, AbstractType<?> newType)
 +    {
 +        List<AbstractType<?>> newTypes = new ArrayList<AbstractType<?>>(types);
 +        newTypes.set(position, newType);
 +        return new CompoundCType(newTypes);
 +    }
 +
 +    public AbstractType<?> asAbstractType()
 +    {
 +        return CompositeType.getInstance(types);
 +    }
 +
 +    public static class CompoundCBuilder implements CBuilder
 +    {
 +        private final CType type;
 +        private final ByteBuffer[] values;
 +        private int size;
 +        private boolean built;
 +
 +        public CompoundCBuilder(CType type)
 +        {
 +            this.type = type;
 +            this.values = new ByteBuffer[type.size()];
 +        }
 +
 +        public int remainingCount()
 +        {
 +            return values.length - size;
 +        }
 +
 +        public CBuilder add(ByteBuffer value)
 +        {
 +            if (isDone())
 +                throw new IllegalStateException();
 +            values[size++] = value;
 +            return this;
 +        }
 +
 +        public CBuilder add(Object value)
 +        {
 +            return add(((AbstractType)type.subtype(size)).decompose(value));
 +        }
 +
 +        private boolean isDone()
 +        {
 +            return remainingCount() == 0 || built;
 +        }
 +
 +        public Composite build()
 +        {
 +            if (size == 0)
 +                return Composites.EMPTY;
 +
 +            // We don't allow to add more element to a builder that has been built so
 +            // that we don't have to copy values.
 +            built = true;
 +
 +            // If the builder is full and we're building a dense cell name, then we can
 +            // directly allocate the CellName object as it's complete.
 +            if (size == values.length && type instanceof CellNameType && ((CellNameType)type).isDense())
 +                return new CompoundDenseCellName(values);
 +            return new CompoundComposite(values, size, false);
 +        }
 +
 +        public Composite buildWith(ByteBuffer value)
 +        {
 +            ByteBuffer[] newValues = Arrays.copyOf(values, values.length);
 +            newValues[size] = value;
 +            // Same as above
 +            if (size+1 == newValues.length && type instanceof CellNameType && ((CellNameType)type).isDense())
 +                return new CompoundDenseCellName(newValues);
 +
 +            return new CompoundComposite(newValues, size+1, false);
 +        }
++
++        public Composite buildWith(List<ByteBuffer> newValues)
++        {
++            ByteBuffer[] buffers = Arrays.copyOf(values, values.length);
++            int newSize = size;
++            for (ByteBuffer value : newValues)
++                buffers[newSize++] = value;
++
++            if (newSize == buffers.length && type instanceof CellNameType && ((CellNameType)type).isDense())
++                return new CompoundDenseCellName(buffers);
++
++            return new CompoundComposite(buffers, newSize, false);
++        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/db/composites/SimpleCType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/SimpleCType.java
index 35e14f9,0000000..229d538
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleCType.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleCType.java
@@@ -1,160 -1,0 +1,170 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
++import java.util.List;
 +
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +/**
 + * A not truly-composite CType.
 + */
 +public class SimpleCType extends AbstractCType
 +{
 +    protected final AbstractType<?> type;
 +
 +    public SimpleCType(AbstractType<?> type)
 +    {
 +        super(type.isByteOrderComparable());
 +        this.type = type;
 +    }
 +
 +    public boolean isCompound()
 +    {
 +        return false;
 +    }
 +
 +    public int size()
 +    {
 +        return 1;
 +    }
 +
 +    public int compare(Composite c1, Composite c2)
 +    {
 +        // This method assumes that simple composites never have an EOC != NONE. This assumption
 +        // stands in particular on the fact that a Composites.EMPTY never has a non-NONE EOC. If
 +        // this ever change, we'll need to update this.
 +
 +        if (isByteOrderComparable)
 +        {
 +            // toByteBuffer is always cheap for simple types, and we keep virtual method calls to a minimum:
 +            // hasRemaining will always be inlined, as will most of the call-stack for BBU.compareUnsigned
 +            ByteBuffer b1 = c1.toByteBuffer();
 +            ByteBuffer b2 = c2.toByteBuffer();
 +            if (!b1.hasRemaining() || !b2.hasRemaining())
 +                return b1.hasRemaining() ? 1 : (b2.hasRemaining() ? -1 : 0);
 +            return ByteBufferUtil.compareUnsigned(b1, b2);
 +        }
 +
 +        boolean c1isEmpty = c1.isEmpty();
 +        boolean c2isEmpty = c2.isEmpty();
 +        if (c1isEmpty || c2isEmpty)
 +            return !c1isEmpty ? 1 : (!c2isEmpty ? -1 : 0);
 +
 +        return type.compare(c1.get(0), c2.get(0));
 +    }
 +
 +    public AbstractType<?> subtype(int i)
 +    {
 +        if (i != 0)
 +            throw new IndexOutOfBoundsException();
 +        return type;
 +    }
 +
 +    public Composite fromByteBuffer(ByteBuffer bytes)
 +    {
 +        return !bytes.hasRemaining() ? Composites.EMPTY : new SimpleComposite(bytes);
 +    }
 +
 +    public CBuilder builder()
 +    {
 +        return new SimpleCBuilder(this);
 +    }
 +
 +    public CType setSubtype(int position, AbstractType<?> newType)
 +    {
 +        if (position != 0)
 +            throw new IndexOutOfBoundsException();
 +        return new SimpleCType(newType);
 +    }
 +
 +    // Use sparingly, it defeats the purpose
 +    public AbstractType<?> asAbstractType()
 +    {
 +        return type;
 +    }
 +
 +    public static class SimpleCBuilder implements CBuilder
 +    {
 +        private final CType type;
 +        private ByteBuffer value;
 +
 +        public SimpleCBuilder(CType type)
 +        {
 +            this.type = type;
 +        }
 +
 +        public int remainingCount()
 +        {
 +            return value == null ? 1 : 0;
 +        }
 +
 +        public CBuilder add(ByteBuffer value)
 +        {
 +            if (this.value != null)
 +                throw new IllegalStateException();
 +            this.value = value;
 +            return this;
 +        }
 +
 +        public CBuilder add(Object value)
 +        {
 +            return add(((AbstractType)type.subtype(0)).decompose(value));
 +        }
 +
 +        public Composite build()
 +        {
 +            if (value == null || !value.hasRemaining())
 +                return Composites.EMPTY;
 +
 +            // If we're building a dense cell name, then we can directly allocate the
 +            // CellName object as it's complete.
 +            if (type instanceof CellNameType && ((CellNameType)type).isDense())
 +                return new SimpleDenseCellName(value);
 +
 +            return new SimpleComposite(value);
 +        }
 +
 +        public Composite buildWith(ByteBuffer value)
 +        {
 +            if (this.value != null)
 +                throw new IllegalStateException();
 +
 +            if (value == null || !value.hasRemaining())
 +                return Composites.EMPTY;
 +
 +            // If we're building a dense cell name, then we can directly allocate the
 +            // CellName object as it's complete.
 +            if (type instanceof CellNameType && ((CellNameType)type).isDense())
 +                return new SimpleDenseCellName(value);
 +
 +            return new SimpleComposite(value);
 +        }
++
++        public Composite buildWith(List<ByteBuffer> values)
++        {
++            if (values.size() > 1)
++                throw new IllegalStateException();
++            if (values.isEmpty())
++                return Composites.EMPTY;
++            return buildWith(values.get(0));
++        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
index 0000000,b728cba..35646be
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
@@@ -1,0 -1,1112 +1,1114 @@@
+ /*
+  * 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 org.apache.cassandra.SchemaLoader;
++import org.apache.cassandra.cql3.statements.ParsedStatement;
+ import org.apache.cassandra.db.ConsistencyLevel;
+ import org.apache.cassandra.db.marshal.*;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.exceptions.RequestExecutionException;
+ import org.apache.cassandra.exceptions.RequestValidationException;
+ import org.apache.cassandra.exceptions.SyntaxException;
+ import org.apache.cassandra.gms.Gossiper;
++import org.apache.cassandra.serializers.CollectionSerializer;
+ import org.apache.cassandra.service.ClientState;
+ import org.apache.cassandra.service.QueryState;
+ import org.apache.cassandra.transport.messages.ResultMessage;
+ import org.apache.cassandra.utils.ByteBufferUtil;
+ import org.apache.cassandra.utils.MD5Digest;
+ import org.junit.AfterClass;
+ import org.junit.BeforeClass;
+ import org.junit.Test;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.*;
+ 
++import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
+ import static org.apache.cassandra.cql3.QueryProcessor.process;
 -import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
+ import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.assertEquals;
+ import static com.google.common.collect.Lists.newArrayList;
+ import static org.junit.Assert.fail;
+ 
+ public class MultiColumnRelationTest
+ {
+     private static final Logger logger = LoggerFactory.getLogger(MultiColumnRelationTest.class);
+     static ClientState clientState;
+     static String keyspace = "multi_column_relation_test";
+ 
+     @BeforeClass
+     public static void setUpClass() throws Throwable
+     {
+         SchemaLoader.loadSchema();
+         executeSchemaChange("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
+         executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_partition (a int PRIMARY KEY, b int)");
+         executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.compound_partition (a int, b int, c int, PRIMARY KEY ((a, b)))");
+         executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_clustering (a int, b int, c int, PRIMARY KEY (a, b))");
+         executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.multiple_clustering (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))");
+         executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.multiple_clustering_reversed (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
+         clientState = ClientState.forInternalCalls();
+     }
+ 
+     @AfterClass
+     public static void stopGossiper()
+     {
+         Gossiper.instance.stop();
+     }
+ 
+     private static void executeSchemaChange(String query) throws Throwable
+     {
+         try
+         {
+             process(String.format(query, keyspace), ConsistencyLevel.ONE);
+         } catch (RuntimeException exc)
+         {
+             throw exc.getCause();
+         }
+     }
+ 
+     private static UntypedResultSet execute(String query) throws Throwable
+     {
+         try
+         {
 -            return processInternal(String.format(query, keyspace));
++            return executeOnceInternal(String.format(query, keyspace));
+         } catch (RuntimeException exc)
+         {
+             if (exc.getCause() != null)
+                 throw exc.getCause();
+             throw exc;
+         }
+     }
+ 
+     private MD5Digest prepare(String query) throws RequestValidationException
+     {
+         ResultMessage.Prepared prepared = QueryProcessor.prepare(String.format(query, keyspace), clientState, false);
+         return prepared.statementId;
+     }
+ 
+     private UntypedResultSet executePrepared(MD5Digest statementId, QueryOptions options) throws RequestValidationException, RequestExecutionException
+     {
 -        CQLStatement statement = QueryProcessor.instance.getPrepared(statementId);
 -        ResultMessage message = statement.executeInternal(QueryState.forInternalCalls(), options);
++        ParsedStatement.Prepared prepared = QueryProcessor.instance.getPrepared(statementId);
++        ResultMessage message = prepared.statement.executeInternal(QueryState.forInternalCalls(), options);
+ 
+         if (message instanceof ResultMessage.Rows)
 -            return new UntypedResultSet(((ResultMessage.Rows)message).result);
++            return UntypedResultSet.create(((ResultMessage.Rows)message).result);
+         else
+             return null;
+     }
+ 
+     @Test(expected=SyntaxException.class)
+     public void testEmptyIdentifierTuple() throws Throwable
+     {
+         execute("SELECT * FROM %s.single_clustering WHERE () = (1, 2)");
+     }
+ 
+     @Test(expected=SyntaxException.class)
+     public void testEmptyValueTuple() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > ()");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testDifferentTupleLengths() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (1, 2, 3)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testNullInTuple() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (1, null)");
+     }
+ 
+     @Test
+     public void testEmptyIN() throws Throwable
+     {
+         UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ()");
+         assertTrue(results.isEmpty());
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testNullInINValues() throws Throwable
+     {
+         UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((1, 2, null))");
+         assertTrue(results.isEmpty());
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPartitionKeyInequality() throws Throwable
+     {
+         execute("SELECT * FROM %s.single_partition WHERE (a) > (1)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPartitionKeyEquality() throws Throwable
+     {
+         execute("SELECT * FROM %s.single_partition WHERE (a) = (0)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testRestrictNonPrimaryKey() throws Throwable
+     {
+         execute("SELECT * FROM %s.single_partition WHERE (b) = (0)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testMixEqualityAndInequality() throws Throwable
+     {
+         execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (0) AND (b) > (0)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testMixMultipleInequalitiesOnSameBound() throws Throwable
+     {
+         execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0) AND (b) > (1)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testClusteringColumnsOutOfOrderInInequality() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) > (0, 0, 0)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testSkipClusteringColumnInEquality() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) = (0, 0)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testSkipClusteringColumnInInequality() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) > (0, 0)");
+     }
+ 
+     @Test
+     public void testSingleClusteringColumnEquality() throws Throwable
+     {
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)");
+         UntypedResultSet results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (1)");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0);
+ 
+         results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (3)");
+         assertEquals(0, results.size());
+     }
+ 
+     @Test
+     public void testMultipleClusteringColumnEquality() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 2, 0, 0)");
+         UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) = (1)");
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(2, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) = (1, 1)");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 1, 0);
+         checkRow(1, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) = (1, 1, 1)");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 1, 1);
+         execute("DELETE FROM %s.multiple_clustering WHERE a=0 AND b=2 and c=0 and d=0");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPartitionAndClusteringColumnEquality() throws Throwable
+     {
+         execute("SELECT * FROM %s.single_clustering WHERE (a, b) = (0, 0)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testClusteringColumnsOutOfOrderInEquality() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) = (3, 2, 1)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testBadType() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) = (1, 2, 'foobar')");
+     }
+ 
+     @Test(expected=SyntaxException.class)
+     public void testSingleColumnTupleRelation() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND b = (1, 2, 3)");
+     }
+ 
+     @Test
+     public void testMixSingleAndTupleInequalities() throws Throwable
+     {
+         String[] queries = new String[]{
+             "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND b < 1",
+             "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND c < 1",
+             "SELECT * FROM %s.multiple_clustering WHERE a=0 AND b > 1 AND (b, c, d) < (1, 1, 0)",
+             "SELECT * FROM %s.multiple_clustering WHERE a=0 AND c > 1 AND (b, c, d) < (1, 1, 0)",
+         };
+ 
+         for (String query : queries)
+         {
+             try
+             {
+                 execute(query);
+                 fail(String.format("Expected query \"%s\" to throw an InvalidRequestException", query));
+             }
+             catch (InvalidRequestException e) {}
+         }
+     }
+ 
+     @Test
+     public void testSingleClusteringColumnInequality() throws Throwable
+     {
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)");
+ 
+         UntypedResultSet results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0)");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 0);
+         checkRow(1, results, 0, 2, 0);
+ 
+         results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= (1)");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 0);
+         checkRow(1, results, 0, 2, 0);
+ 
+         results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < (2)");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 0);
+         checkRow(1, results, 0, 1, 0);
+ 
+         results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= (1)");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 0);
+         checkRow(1, results, 0, 1, 0);
+ 
+         results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0) AND (b) < (2)");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0);
+     }
+ 
+     @Test
+     public void testMultipleClusteringColumnInequality() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+ 
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)");
+ 
+         UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (0)");
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(2, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) >= (0)");
+         assertEquals(6, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+         checkRow(3, results, 0, 1, 0, 0);
+         checkRow(4, results, 0, 1, 1, 0);
+         checkRow(5, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (1, 0)");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 1, 0);
+         checkRow(1, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) >= (1, 0)");
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(2, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (1, 1, 0)");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) >= (1, 1, 0)");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 1, 0);
+         checkRow(1, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) < (1)");
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) <= (1)");
+         assertEquals(6, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+         checkRow(3, results, 0, 1, 0, 0);
+         checkRow(4, results, 0, 1, 1, 0);
+         checkRow(5, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) < (0, 1)");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) <= (0, 1)");
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) < (0, 1, 1)");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) <= (0, 1, 1)");
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND (b) < (1)");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c) < (1, 1)");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c, d) < (1, 1, 0)");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+ 
+         // reversed
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (0) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(3, results.size());
+         checkRow(2, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) >= (0) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(6, results.size());
+         checkRow(5, results, 0, 0, 0, 0);
+         checkRow(4, results, 0, 0, 1, 0);
+         checkRow(3, results, 0, 0, 1, 1);
+         checkRow(2, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (1, 0) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(2, results.size());
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) >= (1, 0) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(3, results.size());
+         checkRow(2, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (1, 1, 0) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) >= (1, 1, 0) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(2, results.size());
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) < (1) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(3, results.size());
+         checkRow(2, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(0, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) <= (1) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(6, results.size());
+         checkRow(5, results, 0, 0, 0, 0);
+         checkRow(4, results, 0, 0, 1, 0);
+         checkRow(3, results, 0, 0, 1, 1);
+         checkRow(2, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) < (0, 1) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) <= (0, 1) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(3, results.size());
+         checkRow(2, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(0, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) < (0, 1, 1) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(2, results.size());
+         checkRow(1, results, 0, 0, 0, 0);
+         checkRow(0, results, 0, 0, 1, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) <= (0, 1, 1) ORDER BY b DESC, c DESC, d DESC");
+         checkRow(2, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(0, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND (b) < (1) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c) < (1, 1) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c, d) < (1, 1, 0) ORDER BY b DESC, c DESC, d DESC");
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+     }
+ 
+     @Test
+     public void testMultipleClusteringColumnInequalityReversedComponents() throws Throwable
+     {
+         // b and d are reversed in the clustering order
+         execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 1, 1)");
+         execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 1, 0)");
+ 
+         execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 1, 1)");
+         execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 1, 0)");
+ 
+ 
+         UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) > (0)");
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 1);
+         checkRow(2, results, 0, 1, 1, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) >= (0)");
+         assertEquals(6, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 1);
+         checkRow(2, results, 0, 1, 1, 0);
+         checkRow(3, results, 0, 0, 0, 0);
+         checkRow(4, results, 0, 0, 1, 1);
+         checkRow(5, results, 0, 0, 1, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) < (1)");
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+         checkRow(2, results, 0, 0, 1, 0);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) <= (1)");
+         assertEquals(6, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 1);
+         checkRow(2, results, 0, 1, 1, 0);
+         checkRow(3, results, 0, 0, 0, 0);
+         checkRow(4, results, 0, 0, 1, 1);
+         checkRow(5, results, 0, 0, 1, 0);
+ 
+         // preserve pre-6875 behavior (even though the query result is technically incorrect)
+         results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b, c) > (1, 0)");
+         assertEquals(0, results.size());
+     }
+ 
+     @Test
+     public void testLiteralIn() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+ 
+         UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 0), (0, 1, 1))");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         // same query, but reversed order for the IN values
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 1), (0, 1, 0))");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN ((0, 1))");
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ((0))");
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testLiteralInWithShortTuple() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1))");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testLiteralInWithLongTuple() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 2, 3, 4))");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testLiteralInWithPartitionKey() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE (a, b, c, d) IN ((0, 1, 2, 3))");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testLiteralInSkipsClusteringColumn() throws Throwable
+     {
+         execute("SELECT * FROM %s.multiple_clustering WHERE (c, d) IN ((0, 1))");
+     }
+     @Test
+     public void testPartitionAndClusteringInClauses() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+ 
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 1, 1)");
+ 
+         UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) AND (b, c, d) IN ((0, 1, 0), (0, 1, 1))");
+         assertEquals(4, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+         checkRow(2, results, 1, 0, 1, 0);
+         checkRow(3, results, 1, 0, 1, 1);
+ 
+         // same query, but reversed order for the IN values
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (1, 0) AND (b, c, d) IN ((0, 1, 1), (0, 1, 0))");
+         assertEquals(4, results.size());
+         checkRow(0, results, 1, 0, 1, 0);
+         checkRow(1, results, 1, 0, 1, 1);
+         checkRow(2, results, 0, 0, 1, 0);
+         checkRow(3, results, 0, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) and (b, c) IN ((0, 1))");
+         assertEquals(4, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+         checkRow(2, results, 1, 0, 1, 0);
+         checkRow(3, results, 1, 0, 1, 1);
+ 
+         results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) and (b) IN ((0))");
+         assertEquals(6, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+         checkRow(3, results, 1, 0, 0, 0);
+         checkRow(4, results, 1, 0, 1, 0);
+         checkRow(5, results, 1, 0, 1, 1);
+     }
+ 
+     // prepare statement tests
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPreparePartitionAndClusteringColumnEquality() throws Throwable
+     {
+         prepare("SELECT * FROM %s.single_clustering WHERE (a, b) = (?, ?)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareDifferentTupleLengths() throws Throwable
+     {
+         prepare("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (?, ?, ?)");
+     }
+ 
+     @Test
+     public void testPrepareEmptyIN() throws Throwable
+     {
+         MD5Digest id = prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ()");
+         UntypedResultSet results = executePrepared(id, makeIntOptions());
+         assertTrue(results.isEmpty());
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPreparePartitionKeyInequality() throws Throwable
+     {
+         prepare("SELECT * FROM %s.single_partition WHERE (a) > (?)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPreparePartitionKeyEquality() throws Throwable
+     {
+         prepare("SELECT * FROM %s.single_partition WHERE (a) = (?)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareRestrictNonPrimaryKey() throws Throwable
+     {
+         prepare("SELECT * FROM %s.single_partition WHERE (b) = (?)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareMixEqualityAndInequality() throws Throwable
+     {
+         prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (?) AND (b) > (?)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareMixMultipleInequalitiesOnSameBound() throws Throwable
+     {
+         prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?) AND (b) > (?)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareClusteringColumnsOutOfOrderInInequality() throws Throwable
+     {
+         prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) > (?, ?, ?)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareSkipClusteringColumnInEquality() throws Throwable
+     {
+         prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) = (?, ?)");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareSkipClusteringColumnInInequality() throws Throwable
+     {
+         prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) > (?, ?)");
+     }
+ 
+     @Test
+     public void testPreparedClusteringColumnEquality() throws Throwable
+     {
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+         MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (?)");
+         UntypedResultSet results = executePrepared(id, makeIntOptions(0));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 0, 0);
+     }
+ 
+     @Test
+     public void testPreparedClusteringColumnEqualitySingleMarker() throws Throwable
+     {
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+         MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = ?");
+         UntypedResultSet results = executePrepared(id, options(tuple(0)));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 0, 0);
+     }
+ 
+     @Test
+     public void testPreparedSingleClusteringColumnInequality() throws Throwable
+     {
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)");
+ 
+         MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?)");
+         UntypedResultSet results = executePrepared(id, makeIntOptions(0));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 0);
+         checkRow(1, results, 0, 2, 0);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= (?)"), makeIntOptions(1));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 0);
+         checkRow(1, results, 0, 2, 0);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < (?)"), makeIntOptions(2));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 0);
+         checkRow(1, results, 0, 1, 0);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= (?)"), makeIntOptions(1));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 0);
+         checkRow(1, results, 0, 1, 0);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?) AND (b) < (?)"), makeIntOptions(0, 2));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0);
+     }
+ 
+     @Test
+     public void testPreparedSingleClusteringColumnInequalitySingleMarker() throws Throwable
+     {
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)");
+         execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)");
+ 
+         MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > ?");
+         UntypedResultSet results = executePrepared(id, options(tuple(0)));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 0);
+         checkRow(1, results, 0, 2, 0);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= ?"), options(tuple(1)));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 0);
+         checkRow(1, results, 0, 2, 0);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < ?"), options(tuple(2)));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 0);
+         checkRow(1, results, 0, 1, 0);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= ?"), options(tuple(1)));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 0);
+         checkRow(1, results, 0, 1, 0);
+ 
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > ? AND (b) < ?"),
+                 options(tuple(0), tuple(2)));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0);
+     }
+ 
+     @Test
+     public void testPrepareMultipleClusteringColumnInequality() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+ 
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)");
+ 
+         UntypedResultSet results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (?)"), makeIntOptions(0));
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(2, results, 0, 1, 1, 1);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (?, ?)"), makeIntOptions(1, 0));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 1, 0);
+         checkRow(1, results, 0, 1, 1, 1);
+ 
+         results = executePrepared(prepare
+                 ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?)"), makeIntOptions(1, 1, 0));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b) < (?)"),
+                 makeIntOptions(0, 1, 0, 1));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 0, 1, 1);
+ 
+         results = executePrepared(prepare
+                 ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)"),
+                 makeIntOptions(0, 1, 1, 1, 1));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)"),
+                 makeIntOptions(0, 1, 1, 1, 1, 0));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+ 
+         // reversed
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (?) ORDER BY b DESC, c DESC, d DESC"),
+                 makeIntOptions(0));
+         assertEquals(3, results.size());
+         checkRow(2, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC"),
+                 makeIntOptions(0, 1, 1, 1, 1));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+     }
+ 
+     @Test
+     public void testPrepareMultipleClusteringColumnInequalitySingleMarker() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+ 
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)");
+ 
+         UntypedResultSet results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > ?"), options(tuple(0)));
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(2, results, 0, 1, 1, 1);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > ?"), options(tuple(1, 0)));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 1, 1, 0);
+         checkRow(1, results, 0, 1, 1, 1);
+ 
+         results = executePrepared(prepare
+                 ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ?"), options(tuple(1, 1, 0)));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b) < ?"),
+                 options(tuple(0, 1, 0), tuple(1)));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 0, 1, 1);
+ 
+         results = executePrepared(prepare
+                 ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c) < ?"),
+                 options(tuple(0, 1, 1), tuple(1, 1)));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c, d) < ?"),
+                 options(tuple(0, 1, 1), tuple(1, 1, 0)));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+ 
+         // reversed
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > ? ORDER BY b DESC, c DESC, d DESC"),
+                 options(tuple(0)));
+         assertEquals(3, results.size());
+         checkRow(2, results, 0, 1, 0, 0);
+         checkRow(1, results, 0, 1, 1, 0);
+         checkRow(0, results, 0, 1, 1, 1);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c) < ? ORDER BY b DESC, c DESC, d DESC"),
+                 options(tuple(0, 1, 1), tuple(1, 1)));
+         assertEquals(1, results.size());
+         checkRow(0, results, 0, 1, 0, 0);
+     }
+ 
+     @Test
+     public void testPrepareLiteralIn() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+ 
+         UntypedResultSet results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))"),
+                 makeIntOptions(0, 1, 0, 0, 1, 1));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         // same query, but reversed order for the IN values
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))"),
+                 makeIntOptions(0, 1, 1, 0, 1, 0));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN ((?, ?))"),
+                 makeIntOptions(0, 1));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ((?))"),
+                 makeIntOptions(0));
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+     }
+ 
+     @Test
+     public void testPrepareInOneMarkerPerTuple() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+ 
+         UntypedResultSet results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN (?, ?)"),
+                 options(tuple(0, 1, 0), tuple(0, 1, 1)));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         // same query, but reversed order for the IN values
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN (?, ?)"),
+                 options(tuple(0, 1, 1), tuple(0, 1, 0)));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN (?)"),
+                 options(tuple(0, 1)));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN (?)"),
+                 options(tuple(0)));
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+     }
+ 
+     @Test
+     public void testPrepareInOneMarker() throws Throwable
+     {
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)");
+         execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)");
+ 
+         UntypedResultSet results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"),
+                 options(list(tuple(0, 1, 0), tuple(0, 1, 1))));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         // same query, but reversed order for the IN values
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"),
+                 options(list(tuple(0, 1, 1), tuple(0, 1, 0))));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         results = executePrepared(prepare(
+                 "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"),
+                 options(list()));
+         assertTrue(results.isEmpty());
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN ?"),
+                 options(list(tuple(0, 1))));
+         assertEquals(2, results.size());
+         checkRow(0, results, 0, 0, 1, 0);
+         checkRow(1, results, 0, 0, 1, 1);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ?"),
+                 options(list(tuple(0))));
+         assertEquals(3, results.size());
+         checkRow(0, results, 0, 0, 0, 0);
+         checkRow(1, results, 0, 0, 1, 0);
+         checkRow(2, results, 0, 0, 1, 1);
+ 
+         results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ?"),
+                 options(list()));
+         assertTrue(results.isEmpty());
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareLiteralInWithShortTuple() throws Throwable
+     {
+         prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?))");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareLiteralInWithLongTuple() throws Throwable
+     {
+         prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareLiteralInWithPartitionKey() throws Throwable
+     {
+         prepare("SELECT * FROM %s.multiple_clustering WHERE (a, b, c, d) IN ((?, ?, ?, ?))");
+     }
+ 
+     @Test(expected=InvalidRequestException.class)
+     public void testPrepareLiteralInSkipsClusteringColumn() throws Throwable
+     {
+         prepare("SELECT * FROM %s.multiple_clustering WHERE (c, d) IN ((?, ?))");
+     }
+ 
+     private static QueryOptions makeIntOptions(Integer... values)
+     {
+         List<ByteBuffer> buffers = new ArrayList<>(values.length);
+         for (int value : values)
+             buffers.add(ByteBufferUtil.bytes(value));
 -        return new QueryOptions(ConsistencyLevel.ONE, buffers);
++        return QueryOptions.forInternalCalls(buffers);
+     }
+ 
+     private static ByteBuffer tuple(Integer... values)
+     {
+         List<AbstractType<?>> types = new ArrayList<>(values.length);
+         ByteBuffer[] buffers = new ByteBuffer[values.length];
+         for (int i = 0; i < values.length; i++)
+         {
+             types.add(Int32Type.instance);
+             buffers[i] = ByteBufferUtil.bytes(values[i]);
+         }
+ 
+         TupleType type = new TupleType(types);
+         return type.buildValue(buffers);
+     }
+ 
+     private static ByteBuffer list(ByteBuffer... values)
+     {
 -        return CollectionType.pack(Arrays.asList(values), values.length);
++        return CollectionSerializer.pack(Arrays.asList(values), values.length, 3);
+     }
+ 
+     private static QueryOptions options(ByteBuffer... buffers)
+     {
 -        return new QueryOptions(ConsistencyLevel.ONE, Arrays.asList(buffers));
++        return QueryOptions.forInternalCalls(Arrays.asList(buffers));
+     }
+ 
+     private static void checkRow(int rowIndex, UntypedResultSet results, Integer... expectedValues)
+     {
+         List<UntypedResultSet.Row> rows = newArrayList(results.iterator());
+         UntypedResultSet.Row row = rows.get(rowIndex);
+         Iterator<ColumnSpecification> columns = row.getColumns().iterator();
+         for (Integer expected : expectedValues)
+         {
+             String columnName = columns.next().name.toString();
+             int actual = row.getInt(columnName);
+             assertEquals(String.format("Expected value %d for column %s in row %d, but got %s", actual, columnName, rowIndex, expected),
+                          (long) expected, actual);
+         }
+     }
+ }


[7/7] git commit: Merge branch 'cassandra-2.1' into trunk

Posted by ty...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: e5ab470d527c86cfa8347739eeb76c9ef1b5aa9b
Parents: 864865d bf52190
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Thu May 22 14:20:49 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Thu May 22 14:20:49 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 .../apache/cassandra/cql3/AbstractMarker.java   |   10 +-
 .../org/apache/cassandra/cql3/Constants.java    |    6 +
 src/java/org/apache/cassandra/cql3/Cql.g        |  109 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |   13 +-
 .../cassandra/cql3/MultiColumnRelation.java     |  144 +++
 .../org/apache/cassandra/cql3/Relation.java     |  104 +-
 .../cassandra/cql3/SingleColumnRelation.java    |   95 ++
 src/java/org/apache/cassandra/cql3/Term.java    |   10 +
 src/java/org/apache/cassandra/cql3/Tuples.java  |  349 ++++++
 .../cql3/statements/ModificationStatement.java  |   20 +-
 .../cql3/statements/MultiColumnRestriction.java |  137 +++
 .../cassandra/cql3/statements/Restriction.java  |  395 +------
 .../cql3/statements/SelectStatement.java        |  805 ++++++++-----
 .../statements/SingleColumnRestriction.java     |  413 +++++++
 .../cassandra/db/composites/CBuilder.java       |    2 +
 .../cassandra/db/composites/Composites.java     |    2 +
 .../cassandra/db/composites/CompoundCType.java  |   13 +
 .../cassandra/db/composites/SimpleCType.java    |   10 +
 .../apache/cassandra/db/marshal/TupleType.java  |  279 +++++
 .../cassandra/cql3/MultiColumnRelationTest.java | 1114 ++++++++++++++++++
 21 files changed, 3263 insertions(+), 768 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5ab470d/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5ab470d/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------