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 2015/03/05 19:29:39 UTC

[1/5] cassandra git commit: Fix mixed single and multi-column relation support

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 ecf48dd79 -> 9c7a601bb


http://git-wip-us.apache.org/repos/asf/cassandra/blob/90a012a1/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
new file mode 100644
index 0000000..1a66002
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
@@ -0,0 +1,973 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.CFDefinition.Name;
+import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+
+public class SelectStatementTest
+{
+    @Test
+    public void testBuildBoundWithNoRestrictions() throws Exception
+    {
+        Restriction[] restrictions = new Restriction[2];
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+    }
+
+    /**
+     * Test 'clustering_0 = 1' with only one clustering column
+     */
+    @Test
+    public void testBuildBoundWithOneEqRestrictionsAndOneClusteringColumn() throws Exception
+    {
+        ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
+        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_0), false);
+        Restriction[] restrictions = new Restriction[] { eq };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), clustering_0);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), clustering_0);
+    }
+
+    /**
+     * Test 'clustering_1 = 1' with 2 clustering columns
+     */
+    @Test
+    public void testBuildBoundWithOneEqRestrictionsAndTwoClusteringColumns() throws Exception
+    {
+        ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
+        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_0), false);
+        Restriction[] restrictions = new Restriction[] { eq, null };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), clustering_0);
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertEndOfRangeComposite(cfDef, bounds.get(0), clustering_0);
+    }
+
+    /**
+     * Test 'clustering_0 IN (1, 2, 3)' with only one clustering column
+     */
+    @Test
+    public void testBuildBoundWithOneInRestrictionsAndOneClusteringColumn() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        SingleColumnRestriction.IN in = new SingleColumnRestriction.InWithValues(toTerms(value1, value2, value3));
+        Restriction[] restrictions = new Restriction[] { in };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(3, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1);
+        assertComposite(cfDef, bounds.get(1), value2);
+        assertComposite(cfDef, bounds.get(2), value3);
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(3, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1);
+        assertComposite(cfDef, bounds.get(1), value2);
+        assertComposite(cfDef, bounds.get(2), value3);
+    }
+
+    /**
+     * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
+     */
+    @Test
+    public void testBuildBoundWithSliceRestrictionsAndOneClusteringColumn() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        SingleColumnRestriction.Slice slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GT, toTerm(value1));
+        Restriction[] restrictions = new Restriction[] { slice };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GTE, toTerm(value1));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.LTE, toTerm(value1));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1);
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.LT, toTerm(value1));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1);
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GT, toTerm(value1));
+        slice.setBound(Relation.Type.LT, toTerm(value2));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GTE, toTerm(value1));
+        slice.setBound(Relation.Type.LTE, toTerm(value2));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2);
+    }
+//
+    /**
+     * Test 'clustering_0 = 1 AND clustering_1 IN (1, 2, 3)' with two clustering columns
+     */
+    @Test
+    public void testBuildBoundWithEqAndInRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        SingleColumnRestriction.IN in = new SingleColumnRestriction.InWithValues(toTerms(value1, value2, value3));
+        Restriction[] restrictions = new Restriction[] { eq, in };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(3, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value1);
+        assertComposite(cfDef, bounds.get(1), value1, value2);
+        assertComposite(cfDef, bounds.get(2), value1, value3);
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(3, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value1);
+        assertComposite(cfDef, bounds.get(1), value1, value2);
+        assertComposite(cfDef, bounds.get(2), value1, value3);
+    }
+
+    /**
+     * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
+     */
+    @Test
+    public void testBuildBoundWithEqAndSliceRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+
+        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(value3), false);
+
+        SingleColumnRestriction.Slice slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GT, toTerm(value1));
+        Restriction[] restrictions = new Restriction[] { eq, slice };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value3, value1);
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertEndOfRangeComposite(cfDef, bounds.get(0), value3);
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GTE, toTerm(value1));
+        restrictions = new Restriction[] { eq, slice };
+
+        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value3, value1);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertEndOfRangeComposite(cfDef, bounds.get(0), value3);
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.LTE, toTerm(value1));
+        restrictions = new Restriction[] { eq, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value3, value1);
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.LT, toTerm(value1));
+        restrictions = new Restriction[] { eq, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value3, value1);
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GT, toTerm(value1));
+        slice.setBound(Relation.Type.LT, toTerm(value2));
+        restrictions = new Restriction[] { eq, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value3, value1);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value3, value2);
+
+        slice = new SingleColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GTE, toTerm(value1));
+        slice.setBound(Relation.Type.LTE, toTerm(value2));
+        restrictions = new Restriction[] { eq, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value3, value1);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value3, value2);
+    }
+
+    /**
+     * Test '(clustering_0, clustering_1) = (1, 2)' with two clustering column
+     */
+    @Test
+    public void testBuildBoundWithMultiEqRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        MultiColumnRestriction.EQ eq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+        Restriction[] restrictions = new Restriction[] { eq, eq };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2);
+    }
+
+    /**
+     * Test '(clustering_0, clustering_1) IN ((1, 2), (2, 3))' with two clustering column
+     */
+    @Test
+    public void testBuildBoundWithMultiInRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        List<Term> terms = asList(toMultiItemTerminal(value1, value2), toMultiItemTerminal(value2, value3));
+        MultiColumnRestriction.IN in = new MultiColumnRestriction.InWithValues(terms);
+        Restriction[] restrictions = new Restriction[] { in, in };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2);
+        assertComposite(cfDef, bounds.get(1), value2, value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2);
+        assertComposite(cfDef, bounds.get(1), value2, value3);
+    }
+
+    /**
+     * Test multi-column slice restrictions (e.g '(clustering_0) > (1)') with only one clustering column
+     */
+    @Test
+    public void testBuildBoundWithMultiSliceRestrictionsWithOneClusteringColumn() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        MultiColumnRestriction.Slice slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1));
+        Restriction[] restrictions = new Restriction[] { slice };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value1));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1);
+
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.LT, toMultiItemTerminal(value1));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1);
+
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1));
+        slice.setBound(Relation.Type.LT, toMultiItemTerminal(value2));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
+
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1));
+        slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value2));
+        restrictions = new Restriction[] { slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2);
+    }
+
+    /**
+     * Test multi-column slice restrictions (e.g '(clustering_0, clustering_1) > (1, 2)') with two clustering
+     * columns
+     */
+    @Test
+    public void testBuildBoundWithMultiSliceRestrictionsWithTwoClusteringColumn() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        // (clustering_0, clustering1) > (1, 2)
+        MultiColumnRestriction.Slice slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1, value2));
+        Restriction[] restrictions = new Restriction[] { slice, slice };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        // (clustering_0, clustering1) >= (1, 2)
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1, value2));
+        restrictions = new Restriction[] { slice, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        // (clustering_0, clustering1) <= (1, 2)
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value1, value2));
+        restrictions = new Restriction[] { slice, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1, value2);
+
+        // (clustering_0, clustering1) < (1, 2)
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.LT, toMultiItemTerminal(value1, value2));
+        restrictions = new Restriction[] { slice, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0));
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1, value2);
+
+        // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1, value2));
+        slice.setBound(Relation.Type.LT, toMultiItemTerminal(value2));
+        restrictions = new Restriction[] { slice, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
+
+        // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
+        slice = new MultiColumnRestriction.Slice(false);
+        slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1, value2));
+        slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value2, value1));
+        restrictions = new Restriction[] { slice, slice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2, value1);
+    }
+
+    /**
+     * Test mixing single and multi equals restrictions (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3))
+     */
+    @Test
+    public void testBuildBoundWithSingleEqAndMultiEqRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3)
+        SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value2, value3), false);
+        Restriction[] restrictions = new Restriction[] { singleEq, multiEq, multiEq };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+
+        // clustering_0 = 1 AND clustering_1 = 2 AND (clustering_2, clustering_3) = (3, 4)
+        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        SingleColumnRestriction.EQ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value2), false);
+        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value3, value4), false);
+        restrictions = new Restriction[] { singleEq, singleEq2, multiEq, multiEq };
+        cfDef = createCFDefinition(restrictions.length);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+
+        // (clustering_0, clustering_1) = (1, 2) AND clustering_2 = 3
+        singleEq = new SingleColumnRestriction.EQ(toTerm(value3), false);
+        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+        restrictions = new Restriction[] { multiEq, multiEq, singleEq };
+        cfDef = createCFDefinition(restrictions.length);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3) AND clustering_3 = 4
+        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        singleEq2 = new SingleColumnRestriction.EQ(toTerm(value4), false);
+        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value2, value3), false);
+        restrictions = new Restriction[] { singleEq, multiEq, multiEq, singleEq2 };
+        cfDef = createCFDefinition(restrictions.length);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+    }
+
+    /**
+     * Test clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
+     */
+    @Test
+    public void testBuildBoundWithSingleEqAndMultiINRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+        ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
+        SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        MultiColumnRestriction.IN multiIn =
+                new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+                                                               toMultiItemTerminal(value4, value5)));
+
+        Restriction[] restrictions = new Restriction[] { singleEq, multiIn, multiIn };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+        assertComposite(cfDef, bounds.get(1), value1, value4, value5);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+        assertComposite(cfDef, bounds.get(1), value1, value4, value5);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3))
+        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        multiIn = new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+                                                                 toMultiItemTerminal(value4, value5)));
+
+        restrictions = new Restriction[] { singleEq, multiIn, multiIn };
+        cfDef = createCFDefinition(restrictions.length);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+        assertComposite(cfDef, bounds.get(1), value1, value4, value5);
+
+        // clustering_0 = 1 AND clustering_1 = 5 AND (clustering_2, clustering_3) IN ((2, 3), (4, 5))
+        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        SingleColumnRestriction.EQ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value5), false);
+        multiIn = new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+                                                                 toMultiItemTerminal(value4, value5)));
+
+        restrictions = new Restriction[] { singleEq, singleEq2, multiIn, multiIn };
+        cfDef = createCFDefinition(restrictions.length);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value5, value2, value3);
+        assertComposite(cfDef, bounds.get(1), value1, value5, value4, value5);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value5, value2, value3);
+        assertComposite(cfDef, bounds.get(1), value1, value5, value4, value5);
+    }
+
+    /**
+     * Test mixing single equal restrictions with multi-column slice restrictions
+     * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
+     */
+    @Test
+    public void testBuildBoundWithSingleEqAndSliceRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+        ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3)
+        SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        MultiColumnRestriction.Slice multiSlice = new MultiColumnRestriction.Slice(false);
+        multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value2, value3));
+
+        Restriction[] restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertEndOfRangeComposite(cfDef, bounds.get(0), value1);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3) AND (clustering_1) < (4)
+        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        multiSlice = new MultiColumnRestriction.Slice(false);
+        multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value2, value3));
+        multiSlice.setBound(Relation.Type.LT, toMultiItemTerminal(value4));
+
+        restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1, value4);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) => (2, 3) AND (clustering_1, clustering_2) <= (4, 5)
+        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+        multiSlice = new MultiColumnRestriction.Slice(false);
+        multiSlice.setBound(Relation.Type.GTE, toMultiItemTerminal(value2, value3));
+        multiSlice.setBound(Relation.Type.LTE, toMultiItemTerminal(value4, value5));
+
+        restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2, value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1, value4, value5);
+    }
+
+    /**
+     * Test mixing multi equal restrictions with single-column slice restrictions
+     * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
+     */
+    @Test
+    public void testBuildBoundWithMultiEqAndSingleSliceRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+
+        // (clustering_0, clustering_1) = (1, 2) AND clustering_2 > 3
+        MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+        SingleColumnRestriction.Slice singleSlice = new SingleColumnRestriction.Slice(false);
+        singleSlice.setBound(Relation.Type.GT, toTerm(value3));
+
+        Restriction[] restrictions = new Restriction[] { multiEq, multiEq, singleSlice };
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertEndOfRangeComposite(cfDef, bounds.get(0), value1, value2);
+    }
+
+    @Test
+    public void testBuildBoundWithSeveralMultiColumnRestrictions() throws Exception
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+        ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) > (3, 4)
+        MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+        MultiColumnRestriction.Slice multiSlice = new MultiColumnRestriction.Slice(false);
+        multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value3, value4));
+
+        Restriction[] restrictions = new Restriction[] { multiEq, multiEq, multiSlice, multiSlice};
+        CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT,value1, value2, value3, value4);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertEndOfRangeComposite(cfDef, bounds.get(0),  value1, value2);
+
+        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) IN ((3, 4), (4, 5))
+        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+        MultiColumnRestriction.IN multiIn =
+                new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value3, value4),
+                                                               toMultiItemTerminal(value4, value5)));
+
+        restrictions = new Restriction[] { multiEq, multiEq, multiIn, multiIn};
+        cfDef = createCFDefinition(restrictions.length);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+        assertComposite(cfDef, bounds.get(1), value1, value2, value4, value5);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(2, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+        assertComposite(cfDef, bounds.get(1), value1, value2, value4, value5);
+
+        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) = ((3, 4), (4, 5))
+        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+        MultiColumnRestriction.EQ multiEq2 = new MultiColumnRestriction.EQ(toMultiItemTerminal(value3, value4), false);
+
+        restrictions = new Restriction[] { multiEq, multiEq, multiEq2, multiEq2};
+        cfDef = createCFDefinition(restrictions.length);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+
+        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+        assertEquals(1, bounds.size());
+        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+    }
+
+    /**
+     * Asserts that the specified composite contains the specified elements.
+     *
+     * @param cfDef the Column Family Definition
+     * @param actual the buffer to test
+     * @param elements the expected elements of the composite
+     */
+    private static void assertComposite(CFDefinition cfDef, ByteBuffer actual, ByteBuffer... elements)
+    {
+        ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
+        assertArrayEquals("the composite is not the expected one:", actual.array(), builder.build().array());
+    }
+
+    /**
+     * Asserts that the specified composite is an end of range composite that contains the specified elements.
+     *
+     * @param cfDef the Column Family Definition
+     * @param actual the buffer to test
+     * @param elements the expected elements of the composite
+     */
+    private static void assertEndOfRangeComposite(CFDefinition cfDef, ByteBuffer actual, ByteBuffer... elements)
+    {
+        ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
+        assertArrayEquals("the composite is not the expected one:", actual.array(), builder.buildAsEndOfRange().array());
+    }
+
+    /**
+     * Asserts that the specified composite is an end of range composite that contains the specified elements.
+     *
+     * @param cfDef the Column Family Definition
+     * @param actual the buffer to test
+     * @param elements the expected elements of the composite
+     */
+    private static void assertForRelationComposite(CFDefinition cfDef,
+                                                   ByteBuffer actual,
+                                                   Relation.Type relType,
+                                                   ByteBuffer... elements)
+    {
+        ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
+        assertArrayEquals("the composite is not the expected one:", actual.array(), builder.buildForRelation(relType).array());
+    }
+
+    /**
+     * Adds all the specified elements to the specified builder.
+     *
+     * @param builder the builder to add to
+     * @param elements the elements to add
+     * @return the builder
+     */
+    private static ColumnNameBuilder addElements(ColumnNameBuilder builder, ByteBuffer... elements)
+    {
+        for (int i = 0, m = elements.length; i < m; i++)
+            builder.add(elements[i]);
+        return builder;
+    }
+
+    /**
+     * Calls the <code>SelectStatement.buildBound</code> with the specified restrictions.
+     *
+     * @param cfDef the Column Family Definition
+     * @param restrictions the restrictions
+     * @return the result from the method call to <code>SelectStatement.buildBound</code>
+     * @throws InvalidRequestException if buildBound throw an <code>Exception</code>
+     */
+    private static List<ByteBuffer> executeBuildBound(CFDefinition cfDef,
+                                                      Restriction[] restrictions,
+                                                      Bound bound) throws InvalidRequestException
+    {
+        return SelectStatement.buildBound(bound,
+                                          new ArrayList<Name>(cfDef.clusteringColumns()),
+                                          restrictions,
+                                          false,
+                                          cfDef,
+                                          cfDef.getColumnNameBuilder(),
+                                          Collections.<ByteBuffer>emptyList());
+    }
+
+    /**
+     * Creates a <code>CFDefinition</code> to be used in the tests.
+     *
+     * @param numberOfClusteringColumns the number of clustering columns
+     * @return a new a <code>CFDefinition</code> instance
+     * @throws ConfigurationException if the CFDefinition cannot be created
+     */
+    private static CFDefinition createCFDefinition(int numberOfClusteringColumns) throws ConfigurationException
+    {
+        List<AbstractType<?>> types = new ArrayList<>();
+        for (int i = 0, m = numberOfClusteringColumns; i < m; i++)
+            types.add(Int32Type.instance);
+
+        CompositeType cType = CompositeType.getInstance(types);
+        CFMetaData cfMetaData = new CFMetaData("keyspace", "test", ColumnFamilyType.Standard, cType);
+        ByteBuffer partitionKey = ByteBufferUtil.bytes("partitionKey");
+        cfMetaData.addColumnDefinition(ColumnDefinition.partitionKeyDef(partitionKey, Int32Type.instance, 0));
+
+        for (int i = 0, m = numberOfClusteringColumns; i < m; i++)
+        {
+            ByteBuffer name = ByteBufferUtil.bytes("clustering_" + i);
+            cfMetaData.addColumnDefinition(ColumnDefinition.clusteringKeyDef(name, types.get(i), i));
+        }
+        cfMetaData.rebuild();
+        return new CFDefinition(cfMetaData);
+    }
+
+    /**
+     * Converts the specified values into a <code>MultiItemTerminal</code>.
+     *
+     * @param values the values to convert.
+     * @return the term corresponding to the specified values.
+     */
+    private static Term toMultiItemTerminal(ByteBuffer... values)
+    {
+        return new Tuples.Value(values);
+    }
+
+    /**
+     * Converts the specified value into a term.
+     *
+     * @param value the value to convert.
+     * @return the term corresponding to the specified value.
+     */
+    private static Term toTerm(ByteBuffer value)
+    {
+        return new Constants.Value(value);
+    }
+
+    /**
+     * Converts the specified values into a <code>List</code> of terms.
+     *
+     * @param values the values to convert.
+     * @return a <code>List</code> of terms corresponding to the specified values.
+     */
+    private static List<Term> toTerms(ByteBuffer... values)
+    {
+        List<Term> terms = new ArrayList<>();
+        for (ByteBuffer value : values)
+            terms.add(toTerm(value));
+        return terms;
+    }
+}


[3/5] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/9c7a601b/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
index 0000000,1a66002..2a1330d
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
@@@ -1,0 -1,973 +1,965 @@@
+ /*
+  * 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 java.nio.ByteBuffer;
+ import java.util.ArrayList;
 -import java.util.Collections;
+ import java.util.List;
+ 
+ import org.junit.Test;
+ 
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.ColumnDefinition;
+ import org.apache.cassandra.cql3.*;
 -import org.apache.cassandra.cql3.CFDefinition.Name;
++import org.apache.cassandra.cql3.Term.MultiItemTerminal;
+ import org.apache.cassandra.db.ColumnFamilyType;
++import org.apache.cassandra.db.composites.Composite;
++import org.apache.cassandra.db.composites.Composite.EOC;
++import org.apache.cassandra.db.composites.Composites;
++import org.apache.cassandra.db.composites.CompoundSparseCellNameType;
+ import org.apache.cassandra.db.marshal.AbstractType;
 -import org.apache.cassandra.db.marshal.CompositeType;
+ import org.apache.cassandra.db.marshal.Int32Type;
 -import org.apache.cassandra.exceptions.ConfigurationException;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.utils.ByteBufferUtil;
+ 
 -import static org.junit.Assert.assertArrayEquals;
 -
+ import static java.util.Arrays.asList;
+ import static org.junit.Assert.assertEquals;
+ 
+ public class SelectStatementTest
+ {
+     @Test
 -    public void testBuildBoundWithNoRestrictions() throws Exception
++    public void testBuildBoundWithNoRestrictions() throws InvalidRequestException
+     {
+         Restriction[] restrictions = new Restriction[2];
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+     }
+ 
+     /**
+      * Test 'clustering_0 = 1' with only one clustering column
+      */
+     @Test
 -    public void testBuildBoundWithOneEqRestrictionsAndOneClusteringColumn() throws Exception
++    public void testBuildBoundWithOneEqRestrictionsAndOneClusteringColumn() throws InvalidRequestException
+     {
+         ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
+         SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_0), false);
+         Restriction[] restrictions = new Restriction[] { eq };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), clustering_0);
++        assertComposite(bounds.get(0), clustering_0, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), clustering_0);
++        assertComposite(bounds.get(0), clustering_0, EOC.END);
+     }
+ 
+     /**
+      * Test 'clustering_1 = 1' with 2 clustering columns
+      */
+     @Test
 -    public void testBuildBoundWithOneEqRestrictionsAndTwoClusteringColumns() throws Exception
++    public void testBuildBoundWithOneEqRestrictionsAndTwoClusteringColumns() throws InvalidRequestException
+     {
 -        ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
 -        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_0), false);
++        ByteBuffer clustering_2 = ByteBufferUtil.bytes(1);
++        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_2), false);
+         Restriction[] restrictions = new Restriction[] { eq, null };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), clustering_0);
++        assertComposite(bounds.get(0), clustering_2, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertEndOfRangeComposite(cfDef, bounds.get(0), clustering_0);
++        assertComposite(bounds.get(0), clustering_2, EOC.END);
+     }
+ 
+     /**
+      * Test 'clustering_0 IN (1, 2, 3)' with only one clustering column
+      */
+     @Test
 -    public void testBuildBoundWithOneInRestrictionsAndOneClusteringColumn() throws Exception
++    public void testBuildBoundWithOneInRestrictionsAndOneClusteringColumn() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
+         SingleColumnRestriction.IN in = new SingleColumnRestriction.InWithValues(toTerms(value1, value2, value3));
+         Restriction[] restrictions = new Restriction[] { in };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(3, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1);
 -        assertComposite(cfDef, bounds.get(1), value2);
 -        assertComposite(cfDef, bounds.get(2), value3);
++        assertComposite(bounds.get(0), value1, EOC.START);
++        assertComposite(bounds.get(1), value2, EOC.START);
++        assertComposite(bounds.get(2), value3, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(3, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1);
 -        assertComposite(cfDef, bounds.get(1), value2);
 -        assertComposite(cfDef, bounds.get(2), value3);
++        assertComposite(bounds.get(0), value1, EOC.END);
++        assertComposite(bounds.get(1), value2, EOC.END);
++        assertComposite(bounds.get(2), value3, EOC.END);
+     }
+ 
+     /**
+      * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
+      */
+     @Test
 -    public void testBuildBoundWithSliceRestrictionsAndOneClusteringColumn() throws Exception
++    public void testBuildBoundWithSliceRestrictionsAndOneClusteringColumn() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ 
+         SingleColumnRestriction.Slice slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GT, toTerm(value1));
++        slice.setBound(Operator.GT, toTerm(value1));
+         Restriction[] restrictions = new Restriction[] { slice };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
++        assertComposite(bounds.get(0), value1, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GTE, toTerm(value1));
++        slice.setBound(Operator.GTE, toTerm(value1));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
++        assertComposite(bounds.get(0), value1, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.LTE, toTerm(value1));
++        slice.setBound(Operator.LTE, toTerm(value1));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1);
++        assertComposite(bounds.get(0), value1, EOC.END);
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.LT, toTerm(value1));
++        slice.setBound(Operator.LT, toTerm(value1));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1);
++        assertComposite(bounds.get(0), value1, EOC.START);
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GT, toTerm(value1));
 -        slice.setBound(Relation.Type.LT, toTerm(value2));
++        slice.setBound(Operator.GT, toTerm(value1));
++        slice.setBound(Operator.LT, toTerm(value2));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
++        assertComposite(bounds.get(0), value1, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
++        assertComposite(bounds.get(0), value2, EOC.START);
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GTE, toTerm(value1));
 -        slice.setBound(Relation.Type.LTE, toTerm(value2));
++        slice.setBound(Operator.GTE, toTerm(value1));
++        slice.setBound(Operator.LTE, toTerm(value1));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
++        assertComposite(bounds.get(0), value1, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2);
++        assertComposite(bounds.get(0), value1, EOC.END);
+     }
 -//
++
+     /**
+      * Test 'clustering_0 = 1 AND clustering_1 IN (1, 2, 3)' with two clustering columns
+      */
+     @Test
 -    public void testBuildBoundWithEqAndInRestrictions() throws Exception
++    public void testBuildBoundWithEqAndInRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
+         SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         SingleColumnRestriction.IN in = new SingleColumnRestriction.InWithValues(toTerms(value1, value2, value3));
+         Restriction[] restrictions = new Restriction[] { eq, in };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(3, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value1);
 -        assertComposite(cfDef, bounds.get(1), value1, value2);
 -        assertComposite(cfDef, bounds.get(2), value1, value3);
++        assertComposite(bounds.get(0), value1, value1, EOC.START);
++        assertComposite(bounds.get(1), value1, value2, EOC.START);
++        assertComposite(bounds.get(2), value1, value3, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(3, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value1);
 -        assertComposite(cfDef, bounds.get(1), value1, value2);
 -        assertComposite(cfDef, bounds.get(2), value1, value3);
++        assertComposite(bounds.get(0), value1, value1, EOC.END);
++        assertComposite(bounds.get(1), value1, value2, EOC.END);
++        assertComposite(bounds.get(2), value1, value3, EOC.END);
+     }
+ 
+     /**
+      * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
+      */
+     @Test
 -    public void testBuildBoundWithEqAndSliceRestrictions() throws Exception
++    public void testBuildBoundWithEqAndSliceRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ 
+         SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(value3), false);
+ 
+         SingleColumnRestriction.Slice slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GT, toTerm(value1));
++        slice.setBound(Operator.GT, toTerm(value1));
+         Restriction[] restrictions = new Restriction[] { eq, slice };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value3, value1);
++        assertComposite(bounds.get(0), value3, value1, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertEndOfRangeComposite(cfDef, bounds.get(0), value3);
++        assertComposite(bounds.get(0), value3, EOC.END);
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GTE, toTerm(value1));
++        slice.setBound(Operator.GTE, toTerm(value1));
+         restrictions = new Restriction[] { eq, slice };
+ 
 -        bounds = executeBuildBound(cfDef,restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value3, value1);
++        assertComposite(bounds.get(0), value3, value1, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertEndOfRangeComposite(cfDef, bounds.get(0), value3);
++        assertComposite(bounds.get(0), value3, EOC.END);
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.LTE, toTerm(value1));
++        slice.setBound(Operator.LTE, toTerm(value1));
+         restrictions = new Restriction[] { eq, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value3);
++        assertComposite(bounds.get(0), value3, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value3, value1);
++        assertComposite(bounds.get(0), value3, value1, EOC.END);
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.LT, toTerm(value1));
++        slice.setBound(Operator.LT, toTerm(value1));
+         restrictions = new Restriction[] { eq, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value3);
++        assertComposite(bounds.get(0), value3, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value3, value1);
++        assertComposite(bounds.get(0), value3, value1, EOC.START);
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GT, toTerm(value1));
 -        slice.setBound(Relation.Type.LT, toTerm(value2));
++        slice.setBound(Operator.GT, toTerm(value1));
++        slice.setBound(Operator.LT, toTerm(value2));
+         restrictions = new Restriction[] { eq, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value3, value1);
++        assertComposite(bounds.get(0), value3, value1, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value3, value2);
++        assertComposite(bounds.get(0), value3, value2, EOC.START);
+ 
+         slice = new SingleColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GTE, toTerm(value1));
 -        slice.setBound(Relation.Type.LTE, toTerm(value2));
++        slice.setBound(Operator.GTE, toTerm(value1));
++        slice.setBound(Operator.LTE, toTerm(value1));
+         restrictions = new Restriction[] { eq, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value3, value1);
++        assertComposite(bounds.get(0), value3, value1, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value3, value2);
++        assertComposite(bounds.get(0), value3, value1, EOC.END);
+     }
+ 
+     /**
+      * Test '(clustering_0, clustering_1) = (1, 2)' with two clustering column
+      */
+     @Test
 -    public void testBuildBoundWithMultiEqRestrictions() throws Exception
++    public void testBuildBoundWithMultiEqRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         MultiColumnRestriction.EQ eq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+         Restriction[] restrictions = new Restriction[] { eq, eq };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2);
++        assertComposite(bounds.get(0), value1, value2, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2);
++        assertComposite(bounds.get(0), value1, value2, EOC.END);
+     }
+ 
+     /**
+      * Test '(clustering_0, clustering_1) IN ((1, 2), (2, 3))' with two clustering column
+      */
+     @Test
 -    public void testBuildBoundWithMultiInRestrictions() throws Exception
++    public void testBuildBoundWithMultiInRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
 -        List<Term> terms = asList(toMultiItemTerminal(value1, value2), toMultiItemTerminal(value2, value3));
++        List<MultiItemTerminal> terms = asList(toMultiItemTerminal(value1, value2), toMultiItemTerminal(value2, value3));
+         MultiColumnRestriction.IN in = new MultiColumnRestriction.InWithValues(terms);
+         Restriction[] restrictions = new Restriction[] { in, in };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2);
 -        assertComposite(cfDef, bounds.get(1), value2, value3);
++        assertComposite(bounds.get(0), value1, value2, EOC.START);
++        assertComposite(bounds.get(1), value2, value3, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2);
 -        assertComposite(cfDef, bounds.get(1), value2, value3);
++        assertComposite(bounds.get(0), value1, value2, EOC.END);
++        assertComposite(bounds.get(1), value2, value3, EOC.END);
+     }
+ 
+     /**
+      * Test multi-column slice restrictions (e.g '(clustering_0) > (1)') with only one clustering column
+      */
+     @Test
 -    public void testBuildBoundWithMultiSliceRestrictionsWithOneClusteringColumn() throws Exception
++    public void testBuildBoundWithMultiSliceRestrictionsWithOneClusteringColumn() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ 
+         MultiColumnRestriction.Slice slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1));
++        slice.setBound(Operator.GT, toMultiItemTerminal(value1));
+         Restriction[] restrictions = new Restriction[] { slice };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
++        assertComposite(bounds.get(0), value1, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1));
++        slice.setBound(Operator.GTE, toMultiItemTerminal(value1));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
++        assertComposite(bounds.get(0), value1, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value1));
++        slice.setBound(Operator.LTE, toMultiItemTerminal(value1));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1);
++        assertComposite(bounds.get(0), value1, EOC.END);
+ 
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.LT, toMultiItemTerminal(value1));
++        slice.setBound(Operator.LT, toMultiItemTerminal(value1));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1);
++        assertComposite(bounds.get(0), value1, EOC.START);
+ 
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1));
 -        slice.setBound(Relation.Type.LT, toMultiItemTerminal(value2));
++        slice.setBound(Operator.GT, toMultiItemTerminal(value1));
++        slice.setBound(Operator.LT, toMultiItemTerminal(value2));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
++        assertComposite(bounds.get(0), value1, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
++        assertComposite(bounds.get(0), value2, EOC.START);
+ 
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1));
 -        slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value2));
++        slice.setBound(Operator.GTE, toMultiItemTerminal(value1));
++        slice.setBound(Operator.LTE, toMultiItemTerminal(value2));
+         restrictions = new Restriction[] { slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
++        assertComposite(bounds.get(0), value1, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2);
++        assertComposite(bounds.get(0), value2, EOC.END);
+     }
+ 
+     /**
 -     * Test multi-column slice restrictions (e.g '(clustering_0, clustering_1) > (1, 2)') with two clustering
 -     * columns
++     * Test multi-column slice restrictions (e.g '(clustering_0, clustering_1) > (1, 2)') with only one clustering
++     * column
+      */
+     @Test
 -    public void testBuildBoundWithMultiSliceRestrictionsWithTwoClusteringColumn() throws Exception
++    public void testBuildBoundWithMultiSliceRestrictionsWithTwoClusteringColumn() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ 
+         // (clustering_0, clustering1) > (1, 2)
+         MultiColumnRestriction.Slice slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1, value2));
++        slice.setBound(Operator.GT, toMultiItemTerminal(value1, value2));
+         Restriction[] restrictions = new Restriction[] { slice, slice };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2);
++        assertComposite(bounds.get(0), value1, value2, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
+         // (clustering_0, clustering1) >= (1, 2)
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1, value2));
++        slice.setBound(Operator.GTE, toMultiItemTerminal(value1, value2));
+         restrictions = new Restriction[] { slice, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2);
++        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
+         // (clustering_0, clustering1) <= (1, 2)
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value1, value2));
++        slice.setBound(Operator.LTE, toMultiItemTerminal(value1, value2));
+         restrictions = new Restriction[] { slice, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1, value2);
++        assertComposite(bounds.get(0), value1, value2, EOC.END);
+ 
+         // (clustering_0, clustering1) < (1, 2)
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.LT, toMultiItemTerminal(value1, value2));
++        slice.setBound(Operator.LT, toMultiItemTerminal(value1, value2));
+         restrictions = new Restriction[] { slice, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0));
++        assertEmptyComposite(bounds.get(0));
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1, value2);
++        assertComposite(bounds.get(0), value1, value2, EOC.START);
+ 
+         // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1, value2));
 -        slice.setBound(Relation.Type.LT, toMultiItemTerminal(value2));
++        slice.setBound(Operator.GT, toMultiItemTerminal(value1, value2));
++        slice.setBound(Operator.LT, toMultiItemTerminal(value2));
+         restrictions = new Restriction[] { slice, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2);
++        assertComposite(bounds.get(0), value1, value2, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
++        assertComposite(bounds.get(0), value2, EOC.START);
+ 
+         // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
+         slice = new MultiColumnRestriction.Slice(false);
 -        slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1, value2));
 -        slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value2, value1));
++        slice.setBound(Operator.GTE, toMultiItemTerminal(value1, value2));
++        slice.setBound(Operator.LTE, toMultiItemTerminal(value2, value1));
+         restrictions = new Restriction[] { slice, slice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2);
++        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2, value1);
++        assertComposite(bounds.get(0), value2, value1, EOC.END);
+     }
+ 
+     /**
+      * Test mixing single and multi equals restrictions (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3))
+      */
+     @Test
 -    public void testBuildBoundWithSingleEqAndMultiEqRestrictions() throws Exception
++    public void testBuildBoundWithSingleEqAndMultiEqRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
+         ByteBuffer value4 = ByteBufferUtil.bytes(4);
+ 
+         // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3)
+         SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value2, value3), false);
+         Restriction[] restrictions = new Restriction[] { singleEq, multiEq, multiEq };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+ 
+         // clustering_0 = 1 AND clustering_1 = 2 AND (clustering_2, clustering_3) = (3, 4)
+         singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         SingleColumnRestriction.EQ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value2), false);
+         multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value3, value4), false);
+         restrictions = new Restriction[] { singleEq, singleEq2, multiEq, multiEq };
 -        cfDef = createCFDefinition(restrictions.length);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+ 
+         // (clustering_0, clustering_1) = (1, 2) AND clustering_2 = 3
+         singleEq = new SingleColumnRestriction.EQ(toTerm(value3), false);
+         multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+         restrictions = new Restriction[] { multiEq, multiEq, singleEq };
 -        cfDef = createCFDefinition(restrictions.length);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+ 
+         // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3) AND clustering_3 = 4
+         singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         singleEq2 = new SingleColumnRestriction.EQ(toTerm(value4), false);
+         multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value2, value3), false);
+         restrictions = new Restriction[] { singleEq, multiEq, multiEq, singleEq2 };
 -        cfDef = createCFDefinition(restrictions.length);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+     }
+ 
+     /**
+      * Test clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
+      */
+     @Test
 -    public void testBuildBoundWithSingleEqAndMultiINRestrictions() throws Exception
++    public void testBuildBoundWithSingleEqAndMultiINRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
+         ByteBuffer value4 = ByteBufferUtil.bytes(4);
+         ByteBuffer value5 = ByteBufferUtil.bytes(5);
+ 
+         // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
+         SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         MultiColumnRestriction.IN multiIn =
+                 new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+                                                                toMultiItemTerminal(value4, value5)));
+ 
+         Restriction[] restrictions = new Restriction[] { singleEq, multiIn, multiIn };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
 -        assertComposite(cfDef, bounds.get(1), value1, value4, value5);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
++        assertComposite(bounds.get(1), value1, value4, value5, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
 -        assertComposite(cfDef, bounds.get(1), value1, value4, value5);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
++        assertComposite(bounds.get(1), value1, value4, value5, EOC.END);
+ 
+         // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3))
+         singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         multiIn = new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+                                                                  toMultiItemTerminal(value4, value5)));
+ 
+         restrictions = new Restriction[] { singleEq, multiIn, multiIn };
 -        cfDef = createCFDefinition(restrictions.length);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3);
 -        assertComposite(cfDef, bounds.get(1), value1, value4, value5);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
++        assertComposite(bounds.get(1), value1, value4, value5, EOC.START);
+ 
+         // clustering_0 = 1 AND clustering_1 = 5 AND (clustering_2, clustering_3) IN ((2, 3), (4, 5))
+         singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         SingleColumnRestriction.EQ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value5), false);
+         multiIn = new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+                                                                  toMultiItemTerminal(value4, value5)));
+ 
+         restrictions = new Restriction[] { singleEq, singleEq2, multiIn, multiIn };
 -        cfDef = createCFDefinition(restrictions.length);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value5, value2, value3);
 -        assertComposite(cfDef, bounds.get(1), value1, value5, value4, value5);
++        assertComposite(bounds.get(0), value1, value5, value2, value3, EOC.START);
++        assertComposite(bounds.get(1), value1, value5, value4, value5, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value5, value2, value3);
 -        assertComposite(cfDef, bounds.get(1), value1, value5, value4, value5);
++        assertComposite(bounds.get(0), value1, value5, value2, value3, EOC.END);
++        assertComposite(bounds.get(1), value1, value5, value4, value5, EOC.END);
+     }
+ 
+     /**
+      * Test mixing single equal restrictions with multi-column slice restrictions
+      * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
+      */
+     @Test
 -    public void testBuildBoundWithSingleEqAndSliceRestrictions() throws Exception
++    public void testBuildBoundWithSingleEqAndSliceRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
+         ByteBuffer value4 = ByteBufferUtil.bytes(4);
+         ByteBuffer value5 = ByteBufferUtil.bytes(5);
+ 
+         // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3)
+         SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         MultiColumnRestriction.Slice multiSlice = new MultiColumnRestriction.Slice(false);
 -        multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value2, value3));
++        multiSlice.setBound(Operator.GT, toMultiItemTerminal(value2, value3));
+ 
+         Restriction[] restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertEndOfRangeComposite(cfDef, bounds.get(0), value1);
++        assertComposite(bounds.get(0), value1, EOC.END);
+ 
+         // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3) AND (clustering_1) < (4)
+         singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         multiSlice = new MultiColumnRestriction.Slice(false);
 -        multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value2, value3));
 -        multiSlice.setBound(Relation.Type.LT, toMultiItemTerminal(value4));
++        multiSlice.setBound(Operator.GT, toMultiItemTerminal(value2, value3));
++        multiSlice.setBound(Operator.LT, toMultiItemTerminal(value4));
+ 
+         restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1, value4);
++        assertComposite(bounds.get(0), value1, value4, EOC.START);
+ 
+         // clustering_0 = 1 AND (clustering_1, clustering_2) => (2, 3) AND (clustering_1, clustering_2) <= (4, 5)
+         singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+         multiSlice = new MultiColumnRestriction.Slice(false);
 -        multiSlice.setBound(Relation.Type.GTE, toMultiItemTerminal(value2, value3));
 -        multiSlice.setBound(Relation.Type.LTE, toMultiItemTerminal(value4, value5));
++        multiSlice.setBound(Operator.GTE, toMultiItemTerminal(value2, value3));
++        multiSlice.setBound(Operator.LTE, toMultiItemTerminal(value4, value5));
+ 
+         restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2, value3);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.NONE);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1, value4, value5);
++        assertComposite(bounds.get(0), value1, value4, value5, EOC.END);
+     }
+ 
+     /**
+      * Test mixing multi equal restrictions with single-column slice restrictions
+      * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
+      */
+     @Test
 -    public void testBuildBoundWithMultiEqAndSingleSliceRestrictions() throws Exception
++    public void testBuildBoundWithMultiEqAndSingleSliceRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ 
+         // (clustering_0, clustering_1) = (1, 2) AND clustering_2 > 3
+         MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+         SingleColumnRestriction.Slice singleSlice = new SingleColumnRestriction.Slice(false);
 -        singleSlice.setBound(Relation.Type.GT, toTerm(value3));
++        singleSlice.setBound(Operator.GT, toTerm(value3));
+ 
+         Restriction[] restrictions = new Restriction[] { multiEq, multiEq, singleSlice };
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertEndOfRangeComposite(cfDef, bounds.get(0), value1, value2);
++        assertComposite(bounds.get(0),  value1, value2, EOC.END);
+     }
+ 
+     @Test
 -    public void testBuildBoundWithSeveralMultiColumnRestrictions() throws Exception
++    public void testBuildBoundWithSeveralMultiColumnRestrictions() throws InvalidRequestException
+     {
+         ByteBuffer value1 = ByteBufferUtil.bytes(1);
+         ByteBuffer value2 = ByteBufferUtil.bytes(2);
+         ByteBuffer value3 = ByteBufferUtil.bytes(3);
+         ByteBuffer value4 = ByteBufferUtil.bytes(4);
+         ByteBuffer value5 = ByteBufferUtil.bytes(5);
+ 
+         // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) > (3, 4)
+         MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+         MultiColumnRestriction.Slice multiSlice = new MultiColumnRestriction.Slice(false);
 -        multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value3, value4));
++        multiSlice.setBound(Operator.GT, toMultiItemTerminal(value3, value4));
+ 
+         Restriction[] restrictions = new Restriction[] { multiEq, multiEq, multiSlice, multiSlice};
 -        CFDefinition cfDef = createCFDefinition(restrictions.length);
+ 
 -        List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT,value1, value2, value3, value4);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertEndOfRangeComposite(cfDef, bounds.get(0),  value1, value2);
++        assertComposite(bounds.get(0),  value1, value2, EOC.END);
+ 
+         // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) IN ((3, 4), (4, 5))
+         multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+         MultiColumnRestriction.IN multiIn =
+                 new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value3, value4),
+                                                                toMultiItemTerminal(value4, value5)));
+ 
+         restrictions = new Restriction[] { multiEq, multiEq, multiIn, multiIn};
 -        cfDef = createCFDefinition(restrictions.length);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
 -        assertComposite(cfDef, bounds.get(1), value1, value2, value4, value5);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
++        assertComposite(bounds.get(1), value1, value2, value4, value5, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(2, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
 -        assertComposite(cfDef, bounds.get(1), value1, value2, value4, value5);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
++        assertComposite(bounds.get(1), value1, value2, value4, value5, EOC.END);
+ 
 -        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) = ((3, 4), (4, 5))
++        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) IN ((3, 4), (4, 5))
+         multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+         MultiColumnRestriction.EQ multiEq2 = new MultiColumnRestriction.EQ(toMultiItemTerminal(value3, value4), false);
+ 
+         restrictions = new Restriction[] { multiEq, multiEq, multiEq2, multiEq2};
 -        cfDef = createCFDefinition(restrictions.length);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.START);
++        bounds = executeBuildBound(restrictions, Bound.START);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+ 
 -        bounds = executeBuildBound(cfDef, restrictions, Bound.END);
++        bounds = executeBuildBound(restrictions, Bound.END);
+         assertEquals(1, bounds.size());
 -        assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+     }
+ 
+     /**
 -     * Asserts that the specified composite contains the specified elements.
++     * Asserts that the specified <code>Composite</code> is an empty one.
+      *
 -     * @param cfDef the Column Family Definition
 -     * @param actual the buffer to test
 -     * @param elements the expected elements of the composite
++     * @param composite the composite to check
+      */
 -    private static void assertComposite(CFDefinition cfDef, ByteBuffer actual, ByteBuffer... elements)
++    private static void assertEmptyComposite(Composite composite)
+     {
 -        ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
 -        assertArrayEquals("the composite is not the expected one:", actual.array(), builder.build().array());
++        assertEquals(Composites.EMPTY, composite);
+     }
+ 
+     /**
 -     * Asserts that the specified composite is an end of range composite that contains the specified elements.
++     * Asserts that the specified <code>Composite</code> contains the specified element and the specified EOC.
+      *
 -     * @param cfDef the Column Family Definition
 -     * @param actual the buffer to test
 -     * @param elements the expected elements of the composite
++     * @param composite the composite to check
++     * @param element the expected element of the composite
++     * @param eoc the expected EOC of the composite
+      */
 -    private static void assertEndOfRangeComposite(CFDefinition cfDef, ByteBuffer actual, ByteBuffer... elements)
++    private static void assertComposite(Composite composite, ByteBuffer element, EOC eoc)
+     {
 -        ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
 -        assertArrayEquals("the composite is not the expected one:", actual.array(), builder.buildAsEndOfRange().array());
++        assertComposite(composite, eoc, element);
+     }
+ 
+     /**
 -     * Asserts that the specified composite is an end of range composite that contains the specified elements.
++     * Asserts that the specified <code>Composite</code> contains the 2 specified element and the specified EOC.
+      *
 -     * @param cfDef the Column Family Definition
 -     * @param actual the buffer to test
 -     * @param elements the expected elements of the composite
++     * @param composite the composite to check
++     * @param eoc the expected EOC of the composite
++     * @param elements the expected element of the composite
+      */
 -    private static void assertForRelationComposite(CFDefinition cfDef,
 -                                                   ByteBuffer actual,
 -                                                   Relation.Type relType,
 -                                                   ByteBuffer... elements)
++    private static void assertComposite(Composite composite, ByteBuffer firstElement, ByteBuffer secondElement, EOC eoc)
+     {
 -        ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
 -        assertArrayEquals("the composite is not the expected one:", actual.array(), builder.buildForRelation(relType).array());
++        assertComposite(composite, eoc, firstElement, secondElement);
+     }
+ 
+     /**
 -     * Adds all the specified elements to the specified builder.
++     * Asserts that the specified <code>Composite</code> contains the 3 specified element and the specified EOC.
+      *
 -     * @param builder the builder to add to
 -     * @param elements the elements to add
 -     * @return the builder
++     * @param composite the composite to check
++     * @param firstElement the first expected element of the composite
++     * @param secondElement the second expected element of the composite
++     * @param thirdElement the third expected element of the composite
++     * @param eoc the expected EOC of the composite
++     * @param elements the expected element of the composite
+      */
 -    private static ColumnNameBuilder addElements(ColumnNameBuilder builder, ByteBuffer... elements)
++    private static void assertComposite(Composite composite,
++                                        ByteBuffer firstElement,
++                                        ByteBuffer secondElement,
++                                        ByteBuffer thirdElement,
++                                        EOC eoc)
+     {
 -        for (int i = 0, m = elements.length; i < m; i++)
 -            builder.add(elements[i]);
 -        return builder;
++        assertComposite(composite, eoc, firstElement, secondElement, thirdElement);
+     }
+ 
+     /**
 -     * Calls the <code>SelectStatement.buildBound</code> with the specified restrictions.
++     * Asserts that the specified <code>Composite</code> contains the 4 specified element and the specified EOC.
+      *
 -     * @param cfDef the Column Family Definition
 -     * @param restrictions the restrictions
 -     * @return the result from the method call to <code>SelectStatement.buildBound</code>
 -     * @throws InvalidRequestException if buildBound throw an <code>Exception</code>
++     * @param composite the composite to check
++     * @param firstElement the first expected element of the composite
++     * @param secondElement the second expected element of the composite
++     * @param thirdElement the third expected element of the composite
++     * @param fourthElement the fourth expected element of the composite
++     * @param eoc the expected EOC of the composite
++     * @param elements the expected element of the composite
+      */
 -    private static List<ByteBuffer> executeBuildBound(CFDefinition cfDef,
 -                                                      Restriction[] restrictions,
 -                                                      Bound bound) throws InvalidRequestException
++    private static void assertComposite(Composite composite,
++                                        ByteBuffer firstElement,
++                                        ByteBuffer secondElement,
++                                        ByteBuffer thirdElement,
++                                        ByteBuffer fourthElement,
++                                        EOC eoc)
+     {
 -        return SelectStatement.buildBound(bound,
 -                                          new ArrayList<Name>(cfDef.clusteringColumns()),
 -                                          restrictions,
 -                                          false,
 -                                          cfDef,
 -                                          cfDef.getColumnNameBuilder(),
 -                                          Collections.<ByteBuffer>emptyList());
++        assertComposite(composite, eoc, firstElement, secondElement, thirdElement, fourthElement);
+     }
+ 
+     /**
 -     * Creates a <code>CFDefinition</code> to be used in the tests.
++     * Asserts that the specified <code>Composite</code> contains the specified elements and EOC.
+      *
 -     * @param numberOfClusteringColumns the number of clustering columns
 -     * @return a new a <code>CFDefinition</code> instance
 -     * @throws ConfigurationException if the CFDefinition cannot be created
++     * @param composite the composite to check
++     * @param eoc the expected EOC of the composite
++     * @param elements the expected elements of the composite
+      */
 -    private static CFDefinition createCFDefinition(int numberOfClusteringColumns) throws ConfigurationException
++    private static void assertComposite(Composite composite, EOC eoc, ByteBuffer... elements)
++    {
++        assertEquals("the composite size is not the expected one:", elements.length, composite.size());
++        for (int i = 0, m = elements.length; i < m; i++)
++        {
++            ByteBuffer element = elements[i];
++            assertEquals("the element " + i + " of the composite is not the expected one:", element, composite.get(i));
++        }
++        assertEquals("the EOC of the composite is not the expected one:", eoc, composite.eoc());
++    }
++
++    /**
++     * Calls the <code>SelectStatement.buildBound</code> with the specified restrictions.
++     *
++     * @param restrictions the restrictions
++     * @return the result from the method call to <code>SelectStatement.buildBound</code>
++     * @throws InvalidRequestException if the method call throw an exception
++     */
++    private static List<Composite> executeBuildBound(Restriction[] restrictions,
++                                                     Bound bound) throws InvalidRequestException
+     {
+         List<AbstractType<?>> types = new ArrayList<>();
 -        for (int i = 0, m = numberOfClusteringColumns; i < m; i++)
++
++        for (int i = 0, m = restrictions.length; i < m; i++)
+             types.add(Int32Type.instance);
+ 
 -        CompositeType cType = CompositeType.getInstance(types);
++        CompoundSparseCellNameType cType = new CompoundSparseCellNameType(types);
+         CFMetaData cfMetaData = new CFMetaData("keyspace", "test", ColumnFamilyType.Standard, cType);
 -        ByteBuffer partitionKey = ByteBufferUtil.bytes("partitionKey");
 -        cfMetaData.addColumnDefinition(ColumnDefinition.partitionKeyDef(partitionKey, Int32Type.instance, 0));
+ 
 -        for (int i = 0, m = numberOfClusteringColumns; i < m; i++)
++        List<ColumnDefinition> columnDefs = new ArrayList<>();
++        for (int i = 0, m = restrictions.length; i < m; i++)
+         {
+             ByteBuffer name = ByteBufferUtil.bytes("clustering_" + i);
 -            cfMetaData.addColumnDefinition(ColumnDefinition.clusteringKeyDef(name, types.get(i), i));
++            columnDefs.add(ColumnDefinition.clusteringKeyDef(cfMetaData, name, types.get(i), i));
+         }
 -        cfMetaData.rebuild();
 -        return new CFDefinition(cfMetaData);
++
++        return SelectStatement.buildBound(bound, columnDefs, restrictions, false, cType, QueryOptions.DEFAULT);
+     }
+ 
+     /**
+      * Converts the specified values into a <code>MultiItemTerminal</code>.
+      *
+      * @param values the values to convert.
+      * @return the term corresponding to the specified values.
+      */
 -    private static Term toMultiItemTerminal(ByteBuffer... values)
++    private static MultiItemTerminal toMultiItemTerminal(ByteBuffer... values)
+     {
+         return new Tuples.Value(values);
+     }
+ 
+     /**
+      * Converts the specified value into a term.
+      *
+      * @param value the value to convert.
+      * @return the term corresponding to the specified value.
+      */
+     private static Term toTerm(ByteBuffer value)
+     {
+         return new Constants.Value(value);
+     }
+ 
+     /**
+      * Converts the specified values into a <code>List</code> of terms.
+      *
+      * @param values the values to convert.
+      * @return a <code>List</code> of terms corresponding to the specified values.
+      */
+     private static List<Term> toTerms(ByteBuffer... values)
+     {
+         List<Term> terms = new ArrayList<>();
+         for (ByteBuffer value : values)
+             terms.add(toTerm(value));
+         return terms;
+     }
+ }


[5/5] cassandra 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


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

Branch: refs/heads/cassandra-2.1
Commit: 9c7a601bbd6ea0df72f6acea8d4cda2fd13f949c
Parents: ecf48dd 90a012a
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Thu Mar 5 12:29:26 2015 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Thu Mar 5 12:29:26 2015 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../cql3/statements/MultiColumnRestriction.java |   2 +-
 .../cassandra/cql3/statements/Restriction.java  |   2 +-
 .../cql3/statements/SelectStatement.java        | 426 ++++----
 .../statements/SingleColumnRestriction.java     |  11 +-
 .../cassandra/cql3/MultiColumnRelationTest.java | 198 +++-
 .../cql3/statements/SelectStatementTest.java    | 965 +++++++++++++++++++
 7 files changed, 1386 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9c7a601b/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ea79e22,462a8d1..57dd97e
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,36 -1,6 +1,38 @@@
 -2.0.13:
 +2.1.4
 + * Make SSTableRewriter.abort() more robust to failure (CASSANDRA-8832)
 + * Remove cold_reads_to_omit from STCS (CASSANDRA-8860)
 + * Make EstimatedHistogram#percentile() use ceil instead of floor (CASSANDRA-8883)
 + * Fix top partitions reporting wrong cardinality (CASSANDRA-8834)
 + * Fix rare NPE in KeyCacheSerializer (CASSANDRA-8067)
 + * Pick sstables for validation as late as possible inc repairs (CASSANDRA-8366)
 + * Fix commitlog getPendingTasks to not increment (CASSANDRA-8856)
 + * Fix parallelism adjustment in range and secondary index queries
 +   when the first fetch does not satisfy the limit (CASSANDRA-8856)
 + * Check if the filtered sstables is non-empty in STCS (CASSANDRA-8843)
 + * Upgrade java-driver used for cassandra-stress (CASSANDRA-8842)
 + * Fix CommitLog.forceRecycleAllSegments() memory access error (CASSANDRA-8812)
 + * Improve assertions in Memory (CASSANDRA-8792)
 + * Fix SSTableRewriter cleanup (CASSANDRA-8802)
 + * Introduce SafeMemory for CompressionMetadata.Writer (CASSANDRA-8758)
 + * 'nodetool info' prints exception against older node (CASSANDRA-8796)
 + * Ensure SSTableReader.last corresponds exactly with the file end (CASSANDRA-8750)
 + * Make SSTableWriter.openEarly more robust and obvious (CASSANDRA-8747)
 + * Enforce SSTableReader.first/last (CASSANDRA-8744)
 + * Cleanup SegmentedFile API (CASSANDRA-8749)
 + * Avoid overlap with early compaction replacement (CASSANDRA-8683)
 + * Safer Resource Management++ (CASSANDRA-8707)
 + * Write partition size estimates into a system table (CASSANDRA-7688)
 + * cqlsh: Fix keys() and full() collection indexes in DESCRIBE output
 +   (CASSANDRA-8154)
 + * Show progress of streaming in nodetool netstats (CASSANDRA-8886)
 + * IndexSummaryBuilder utilises offheap memory, and shares data between
 +   each IndexSummary opened from it (CASSANDRA-8757)
 + * markCompacting only succeeds if the exact SSTableReader instances being 
 +   marked are in the live set (CASSANDRA-8689)
 + * cassandra-stress support for varint (CASSANDRA-8882)
 +Merged from 2.0:
+  * Fix regression in mixed single and multi-column relation support for
+    SELECT statements (CASSANDRA-8613)
   * Add ability to limit number of native connections (CASSANDRA-8086)
   * Add offline tool to relevel sstables (CASSANDRA-8301)
   * Preserve stream ID for more protocol errors (CASSANDRA-8848)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9c7a601b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
index 96cb905,f643684..6946c98
--- a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
@@@ -59,7 -58,7 +59,7 @@@ public interface MultiColumnRestrictio
       */
      public static class InWithValues extends SingleColumnRestriction.InWithValues implements MultiColumnRestriction.IN
      {
--        public InWithValues(List<Term> values)
++        public InWithValues(List<? extends Term> values)
          {
              super(values);
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9c7a601b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/Restriction.java
index 659ed95,3d33bde..485fd22
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@@ -68,10 -59,10 +68,10 @@@ public interface Restrictio
          /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
          public boolean isInclusive(Bound b);
  
 -        public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound);
 +        public Operator getRelation(Bound eocBound, Bound inclusiveBound);
  
 -        public IndexOperator getIndexOperator(Bound b);
 +        public Operator getIndexOperator(Bound b);
  
-         public void setBound(ColumnIdentifier name, Operator type, Term t) throws InvalidRequestException;
 -        public void setBound(Relation.Type type, Term t) throws InvalidRequestException;
++        public void setBound(Operator type, Term t) throws InvalidRequestException;
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9c7a601b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 9099ba7,6b3c781..fc5e4f6
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -715,19 -717,37 +716,39 @@@ public class SelectStatement implement
          // we always do a slice for CQL3 tables, so it's ok to ignore them here
          assert !isColumnRange();
  
 -        ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
 -        Iterator<CFDefinition.Name> idIter = cfDef.clusteringColumns().iterator();
 -        while (idIter.hasNext())
 +        CBuilder builder = cfm.comparator.prefixBuilder();
-         Iterator<ColumnDefinition> idIter = cfm.clusteringColumns().iterator();
-         for (Restriction r : columnRestrictions)
++
++        Iterator<ColumnDefinition> columnIter = cfm.clusteringColumns().iterator();
++        while (columnIter.hasNext())
          {
-             ColumnDefinition def = idIter.next();
 -            CFDefinition.Name name = idIter.next();
 -            Restriction r = columnRestrictions[name.position];
++            ColumnDefinition def = columnIter.next();
++            Restriction r = columnRestrictions[def.position()];
              assert r != null && !r.isSlice();
  
              if (r.isEQ())
              {
-                 ByteBuffer val = r.values(options).get(0);
-                 if (val == null)
-                     throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name));
-                 builder.add(val);
 -                List<ByteBuffer> values = r.values(variables);
++                List<ByteBuffer> values = r.values(options);
+                 if (r.isMultiColumn())
+                 {
+                     for (int i = 0, m = values.size(); i < m; i++)
+                     {
++                        ByteBuffer val = values.get(i);
++
+                         if (i != 0)
 -                            name = idIter.next();
++                            columnIter.next();
+ 
 -                        ByteBuffer val = values.get(i);
+                         if (val == null)
 -                            throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", name));
++                            throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name));
+                         builder.add(val);
+                     }
+                 }
+                 else
+                 {
 -                    ByteBuffer val = values.get(0);
++                    ByteBuffer val = r.values(options).get(0);
+                     if (val == null)
 -                        throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", name.name));
++                        throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name));
+                     builder.add(val);
+                 }
              }
              else
              {
@@@ -752,22 -775,32 +773,21 @@@
                      }
                      return columns;
                  }
--                else
++
++                // we have a multi-column IN restriction
++                List<List<ByteBuffer>> values = ((MultiColumnRestriction.IN) r).splitValues(options);
++                TreeSet<CellName> inValues = new TreeSet<>(cfm.comparator);
++                for (List<ByteBuffer> components : values)
                  {
--                    // we have a multi-column IN restriction
-                     List<List<ByteBuffer>> values = ((MultiColumnRestriction.IN) r).splitValues(options);
-                     TreeSet<CellName> inValues = new TreeSet<>(cfm.comparator);
 -                    List<List<ByteBuffer>> values = ((MultiColumnRestriction.IN) r).splitValues(variables);
 -                    if (values.isEmpty())
 -                        return null;
 -                    TreeSet<ByteBuffer> inValues = new TreeSet<>(cfDef.cfm.comparator);
--                    for (List<ByteBuffer> components : values)
--                    {
 -                        ColumnNameBuilder b = builder.copy();
--                        for (int i = 0; i < components.size(); i++)
 -                        {
--                            if (components.get(i) == null)
-                                 throw new InvalidRequestException("Invalid null value in condition for column " + cfm.clusteringColumns().get(i + def.position()));
 -                            {
 -                                List<CFDefinition.Name> clusteringCols = new ArrayList<>(cfDef.clusteringColumns());
 -                                throw new InvalidRequestException("Invalid null value in condition for clustering column " + clusteringCols.get(i + name.position));
 -                            }
 -                            b.add(components.get(i));
 -                        }
 -                        if (cfDef.isCompact)
 -                            inValues.add(b.build());
 -                        else
 -                            inValues.addAll(addSelectedColumns(b));
 -                    }
 -                    return inValues;
++                    for (int i = 0; i < components.size(); i++)
++                        if (components.get(i) == null)
++                            throw new InvalidRequestException("Invalid null value in condition for column "
++                                    + cfm.clusteringColumns().get(i + def.position()).name);
 +
-                         Composite prefix = builder.buildWith(components);
-                         inValues.addAll(addSelectedColumns(prefix));
-                     }
-                     return inValues;
++                    Composite prefix = builder.buildWith(components);
++                    inValues.addAll(addSelectedColumns(prefix));
                  }
++                return inValues;
              }
          }
  
@@@ -827,38 -864,23 +847,24 @@@
          return false;
      }
  
-     private static List<Composite> buildBound(Bound bound,
-                                               List<ColumnDefinition> defs,
-                                               Restriction[] restrictions,
-                                               boolean isReversed,
-                                               CType type,
-                                               QueryOptions options) throws InvalidRequestException
+     @VisibleForTesting
 -    static List<ByteBuffer> buildBound(Bound bound,
 -                                        List<CFDefinition.Name> names,
 -                                        Restriction[] restrictions,
 -                                        boolean isReversed,
 -                                        CFDefinition cfDef,
 -                                        ColumnNameBuilder builder,
 -                                        List<ByteBuffer> variables) throws InvalidRequestException
++    static List<Composite> buildBound(Bound bound,
++                                      List<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 (!defs.isEmpty())
-         {
-             Restriction firstRestriction = restrictions[0];
-             if (firstRestriction != null && firstRestriction.isMultiColumn())
-             {
-                 if (firstRestriction.isSlice())
-                     return buildMultiColumnSliceBound(bound, defs, (MultiColumnRestriction.Slice) firstRestriction, isReversed, builder, options);
-                 else if (firstRestriction.isIN())
-                     return buildMultiColumnInBound(bound, defs, (MultiColumnRestriction.IN) firstRestriction, isReversed, builder, type, options);
-                 else
-                     return buildMultiColumnEQBound(bound, defs, (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),
          // it only depends on whether the slice is reversed
          Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-         for (Iterator<ColumnDefinition> iter = defs.iterator(); iter.hasNext();)
 -        for (int i = 0; i < names.size(); i++)
++        for (int i = 0, m = defs.size(); i < m; i++)
          {
-             ColumnDefinition def = iter.next();
 -            CFDefinition.Name name = names.get(i);
++            ColumnDefinition def = defs.get(i);
  
              // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
              // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
@@@ -875,165 -898,132 +881,140 @@@
              }
              if (r.isSlice())
              {
-                 builder.add(getSliceValue(r, b, options));
+                 if (r.isMultiColumn())
+                 {
 -                    List<ByteBuffer> values = ((MultiColumnRestriction.Slice) r).componentBounds(b, variables);
 -                    List<Name> columns = subList(names, i, values.size());
 -                    addComponents(builder, columns, values);
++                    MultiColumnRestriction.Slice slice = (MultiColumnRestriction.Slice) r;
++
++                    if (!slice.hasBound(b))
++                    {
++                        Composite prefix = builder.build();
++                        return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
++                                ? prefix.end()
++                                : prefix);
++                    }
++
++                    List<ByteBuffer> vals = slice.componentBounds(b, options);
++
++                    for (int j = 0, n = vals.size(); j < n; j++)
++                        addValue(builder, defs.get(i + j), vals.get(j)) ;
+                 }
+                 else
+                 {
 -                    addComponent(builder, name, getSliceValue(r, b, variables));
++                    builder.add(getSliceValue(r, b, options));
+                 }
 -
 -                Relation.Type relType = ((Restriction.Slice)r).getRelation(eocBound, b);
 -                return Collections.singletonList(builder.buildForRelation(relType));
 +                Operator relType = ((Restriction.Slice)r).getRelation(eocBound, b);
 +                return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
              }
-             else
+ 
+             if (r.isIN())
              {
-                 // IN or EQ
+                 // 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);
++                TreeSet<Composite> inValues = new TreeSet<>(isReversed ? type.reverseComparator() : type);
+ 
+                 if (r.isMultiColumn())
+                 {
 -                    List<List<ByteBuffer>> splitInValues = ((MultiColumnRestriction.IN) r).splitValues(variables);
++                    List<List<ByteBuffer>> splitInValues = ((MultiColumnRestriction.IN) r).splitValues(options);
++
+                     for (List<ByteBuffer> components : splitInValues)
+                     {
 -                        ColumnNameBuilder copy = builder.copy();
 -                        List<Name> columns = subList(names, i, components.size());
 -                        addComponents(copy, columns, components);
++                        for (int j = 0; j < components.size(); j++)
++                            if (components.get(j) == null)
++                                throw new InvalidRequestException("Invalid null value in condition for column " + defs.get(i + j).name);
+ 
 -                        inValues.add(buildColumnName(copy, eocBound));
++                        Composite prefix = builder.buildWith(components);
++                        inValues.add(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
+                     }
+                     return new ArrayList<>(inValues);
+                 }
+ 
 -                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 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<>(isReversed ? type.reverseComparator() : type);
                      for (ByteBuffer val : values)
                      {
 -                        ColumnNameBuilder copy = builder.copy();
 -                        addComponent(copy, name, val);
 -                        inValues.add(buildColumnName(copy, eocBound));
 +                        if (val == null)
-                             throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
++                            throw new InvalidRequestException(String.format("Invalid null value in condition for column %s",
++                                                                            def.name));
 +                        Composite prefix = builder.buildWith(val);
 +                        // See below for why this
-                         s.add(builder.remainingCount() == 0 ? prefix : (eocBound == Bound.END ? prefix.end() : prefix.start()));
++                        inValues.add(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
                      }
-                     return new ArrayList<>(s);
+                     return new ArrayList<>(inValues);
                  }
+             }
  
-                 ByteBuffer val = values.get(0);
-                 if (val == null)
-                     throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
-                 builder.add(val);
 -            List<ByteBuffer> values = r.values(variables);
++            List<ByteBuffer> values = r.values(options);
++
+             if (r.isMultiColumn())
+             {
 -                List<Name> columns = subList(names, i, values.size());
 -                addComponents(builder, columns, values);
 -                i += values.size() - 1;
++                for (int j = 0; j < values.size(); j++)
++                    addValue(builder, defs.get(i + j), values.get(j));
++                i += values.size() - 1; // skips the processed columns
+             }
+             else
+             {
 -                addComponent(builder, name, values.get(0));
++                addValue(builder, def, values.get(0));
              }
          }
 -        return Collections.singletonList(buildColumnName(builder, eocBound));
 -    }
 -
 -    /**
 -     * Returns a view of the portion of the list starting at the index offset and containing the number of elements
 -     * specified.
 -     *
 -     * @param list the original list
 -     * @param offset the index offset
 -     * @param length the number of elements
 -     * @return a view of the specified range within this list
 -     */
 -    private static <T> List<T> subList(List<T> list, int offset, int length)
 -    {
 -        return list.subList(offset, offset + length);
 -    }
 -
 -    /**
 -     * Builds the column name when there was not slice.
 -     *
 -     * @param builder the column name builder
 -     * @param eocBound the End of Component bound
 -     * @return the column name
 -     */
 -    private static ByteBuffer buildColumnName(ColumnNameBuilder builder, Bound eocBound)
 -    {
 +        // Means no relation at all or everything was an equal
          // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
          // 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 (eocBound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build();
 +        // prefix.end() will sort after prefix (see #5240).
 +        Composite prefix = builder.build();
-         return Collections.singletonList(builder.remainingCount() == 0 ? prefix : (eocBound == Bound.END ? prefix.end() : prefix.start()));
++        return Collections.singletonList(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
+     }
+ 
+     /**
 -     * Adds the specified component values to the specified builder.
++     * Adds an EOC to the specified Composite.
+      *
 -     * @param builder  the ColumnNameBuilder to which the values must be added
 -     * @param names the column names
 -     * @param values the values to add
 -     * @throws InvalidRequestException if one of the values is null
++     * @param composite the composite
++     * @param eocBound the EOC bound
++     * @return a new <code>Composite</code> with the EOC corresponding to the eocBound
+      */
 -    private static void addComponents(ColumnNameBuilder builder, List<Name> names, List<ByteBuffer> values) throws InvalidRequestException
++    private static Composite addEOC(Composite composite, Bound eocBound)
+     {
 -        for (int i = 0, m = values.size(); i < m; i++)
 -            addComponent(builder, names.get(i), values.get(i));
++        return eocBound == Bound.END ? composite.end() : composite.start();
+     }
+ 
+     /**
 -     * Adds the specified component value to the specified builder
++     * Adds the specified value to the specified builder
+      *
 -     * @param builder the ColumnNameBuilder to which the value must be added
 -     * @param name the column associated to the value
++     * @param builder the CBuilder to which the value must be added
++     * @param def the column associated to the value
+      * @param value the value to add
+      * @throws InvalidRequestException if the value is null
+      */
 -    private static void addComponent(ColumnNameBuilder builder, Name name, ByteBuffer value) throws InvalidRequestException
++    private static void addValue(CBuilder builder, ColumnDefinition def, ByteBuffer value) throws InvalidRequestException
+     {
+         if (value == null)
 -            throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", name));
++            throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", def.name));
+         builder.add(value);
      }
  
 +    private static Composite.EOC eocForRelation(Operator 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 static List<Composite> buildMultiColumnSliceBound(Bound bound,
-                                                               List<ColumnDefinition> defs,
-                                                               MultiColumnRestriction.Slice slice,
-                                                               boolean isReversed,
-                                                               CBuilder builder,
-                                                               QueryOptions options) throws InvalidRequestException
-     {
-         Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
- 
-         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))
-         {
-             Composite prefix = builder.build();
-             return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
-                     ? prefix.end()
-                     : prefix);
-         }
- 
-         List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, options);
- 
-         ByteBuffer v = vals.get(firstName.position());
-         if (v == null)
-             throw new InvalidRequestException("Invalid null value in condition for column " + firstName.name);
-         builder.add(v);
- 
-         while (iter.hasNext())
-         {
-             ColumnDefinition def = iter.next();
-             if (def.position() >= vals.size())
-                 break;
- 
-             v = vals.get(def.position());
-             if (v == null)
-                 throw new InvalidRequestException("Invalid null value in condition for column " + def.name);
-             builder.add(v);
-         }
-         Operator relType = slice.getRelation(eocBound, firstComponentBound);
-         return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
-     }
- 
-     private static List<Composite> buildMultiColumnInBound(Bound bound,
-                                                            List<ColumnDefinition> defs,
-                                                            MultiColumnRestriction.IN restriction,
-                                                            boolean isReversed,
-                                                            CBuilder builder,
-                                                            CType type,
-                                                            QueryOptions options) throws InvalidRequestException
-     {
-         List<List<ByteBuffer>> splitInValues = restriction.splitValues(options);
-         Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
- 
-         // 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> inValues = new TreeSet<>(isReversed ? type.reverseComparator() : type);
-         for (List<ByteBuffer> components : splitInValues)
-         {
-             for (int i = 0; i < components.size(); i++)
-                 if (components.get(i) == null)
-                     throw new InvalidRequestException("Invalid null value in condition for column " + defs.get(i));
- 
-             Composite prefix = builder.buildWith(components);
-             inValues.add(eocBound == Bound.END && builder.remainingCount() - components.size() > 0
-                          ? prefix.end()
-                          : prefix);
-         }
-         return new ArrayList<>(inValues);
-     }
- 
-     private static List<Composite> buildMultiColumnEQBound(Bound bound,
-                                                            List<ColumnDefinition> defs,
-                                                            MultiColumnRestriction.EQ restriction,
-                                                            boolean isReversed,
-                                                            CBuilder builder,
-                                                            QueryOptions options) throws InvalidRequestException
-     {
-         Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-         List<ByteBuffer> values = restriction.values(options);
-         for (int i = 0; i < values.size(); i++)
-         {
-             ByteBuffer component = values.get(i);
-             if (component == null)
-                 throw new InvalidRequestException("Invalid null value in condition for column " + defs.get(i));
-             builder.add(component);
-         }
- 
-         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));
@@@ -1470,12 -1497,6 +1451,10 @@@
               */
              boolean hasQueriableIndex = false;
              boolean hasQueriableClusteringColumnIndex = false;
-             boolean hasSingleColumnRelations = false;
-             boolean hasMultiColumnRelations = false;
 +
 +            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
 +            SecondaryIndexManager indexManager = cfs.indexManager;
 +
              for (Relation relation : whereClause)
              {
                  if (relation.isMultiColumn())
@@@ -1485,12 -1506,11 +1464,11 @@@
                      for (ColumnIdentifier.Raw rawEntity : rel.getEntities())
                      {
                          ColumnIdentifier entity = rawEntity.prepare(cfm);
 -                        boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
 +                        ColumnDefinition def = cfm.getColumnDefinition(entity);
 +                        boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
                          hasQueriableIndex |= queriable[0];
                          hasQueriableClusteringColumnIndex |= queriable[1];
 -                        Name name = cfDef.get(entity);
 -                        names.add(name);
 +                        names.add(def);
-                         hasMultiColumnRelations |= ColumnDefinition.Kind.CLUSTERING_COLUMN.equals(def.kind);
                      }
                      updateRestrictionsForRelation(stmt, names, rel, boundNames);
                  }
@@@ -1498,19 -1518,16 +1476,16 @@@
                  {
                      SingleColumnRelation rel = (SingleColumnRelation) relation;
                      ColumnIdentifier entity = rel.getEntity().prepare(cfm);
 -                    boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
 +                    ColumnDefinition def = cfm.getColumnDefinition(entity);
 +                    boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
                      hasQueriableIndex |= queriable[0];
                      hasQueriableClusteringColumnIndex |= queriable[1];
-                     hasSingleColumnRelations |= ColumnDefinition.Kind.CLUSTERING_COLUMN.equals(def.kind);
 -                    Name name = cfDef.get(entity);
 -                    updateRestrictionsForRelation(stmt, name, rel, boundNames);
 +                    updateRestrictionsForRelation(stmt, def, rel, boundNames);
                  }
              }
-             if (hasSingleColumnRelations && hasMultiColumnRelations)
-                 throw new InvalidRequestException("Mixing single column relations and multi column relations on clustering columns is not allowed");
  
               // 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.
@@@ -1597,47 -1614,82 +1572,83 @@@
              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<>();
+             Restriction existing = null;
  
-             int previousPosition = -1;
-             for (ColumnDefinition def : defs)
 -            int previousPosition = names.get(0).position - 1;
 -            for (int i = 0, m = names.size(); i < m; i++)
++            int previousPosition = defs.get(0).position() - 1;
++            for (int i = 0, m = defs.size(); i < m; i++)
              {
 -                Name name = names.get(i);
++                ColumnDefinition def = defs.get(i);
++
                  // 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));
++                    throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", def.name));
  
 -                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));
++                    throw new InvalidRequestException(String.format("Column \"%s\" appeared twice in a relation: %s", def.name, relation));
 +                seen.add(def);
  
                  // check that no clustering columns were skipped
 -                if (name.position != previousPosition + 1)
 +                if (def.position() != previousPosition + 1)
                  {
                      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(
-                                 "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
+ 
+                     throw new InvalidRequestException(String.format(
 -                            "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
++                                "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s",
++                                 relation));
                  }
                  previousPosition++;
  
-                 Restriction existing = getExistingRestriction(stmt, def);
+                 Restriction previous = existing;
 -                existing = getExistingRestriction(stmt, name);
 -                Relation.Type operator = relation.operator();
++                existing = getExistingRestriction(stmt, def);
 +                Operator operator = relation.operator();
                  if (existing != null)
                  {
 -                    if (operator == Relation.Type.EQ || operator == Relation.Type.IN)
 +                    if (operator == Operator.EQ || operator == Operator.IN)
-                         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()));
+                     {
+                         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()));
++                                def.name, operator));
+                     }
                      else if (!existing.isSlice())
-                         throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", def));
+                     {
+                         throw new InvalidRequestException(String.format(
 -                                "Column \"%s\" cannot be restricted by an equality relation and an inequality relation", name));
++                                "Column \"%s\" cannot be restricted by an equality relation and an inequality relation",
++                                def.name));
+                     }
+                     else
+                     {
+                         if (!existing.isMultiColumn())
+                         {
+                             throw new InvalidRequestException(String.format(
+                                     "Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities: %s",
 -                                    name, relation));
++                                    def.name, relation));
+                         }
+ 
+                         boolean existingRestrictionStartBefore =
 -                                (i == 0 && name.position != 0 && stmt.columnRestrictions[name.position - 1] == existing);
++                            (i == 0 && def.position() != 0 && stmt.columnRestrictions[def.position() - 1] == existing);
+ 
+                         boolean existingRestrictionStartAfter = (i != 0 && previous != existing);
+ 
+                         if (existingRestrictionStartBefore || existingRestrictionStartAfter)
+                         {
+                             throw new InvalidRequestException(String.format(
+                                     "Column \"%s\" cannot be restricted by two tuple-notation inequalities not starting with the same column: %s",
 -                                    name, relation));
++                                    def.name, relation));
+                         }
+ 
 -                        checkBound(existing, name, operator);
++                        checkBound(existing, def, operator);
+                     }
                  }
 -                restrictedColumns.add(name);
 +                restrictedColumns.add(def);
              }
  
 -            boolean onToken = false;
 -
              switch (relation.operator())
              {
                  case EQ:
@@@ -1683,38 -1735,55 +1694,58 @@@
                  case GT:
                  case GTE:
                  {
 -                    Term t = relation.getValue().prepare(names);
 +                    Term t = relation.getValue().prepare(keyspace(), defs);
                      t.collectMarkerSpecification(boundNames);
 -
 -                    Restriction.Slice restriction = (Restriction.Slice) getExistingRestriction(stmt, names.get(0));
++                    Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, defs.get(0));
+                     if (restriction == null)
+                         restriction = new MultiColumnRestriction.Slice(false);
+                     restriction.setBound(relation.operator(), t);
+ 
 -                    for (CFDefinition.Name name : names)
 +                    for (ColumnDefinition def : defs)
                      {
-                         Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, def);
-                         if (restriction == null)
-                             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: %s", def.name, relation));
-                         restriction.setBound(def.name, relation.operator(), t);
 -                        stmt.columnRestrictions[name.position] = restriction;
 +                        stmt.columnRestrictions[def.position()] = restriction;
                      }
 +                    break;
                  }
 +                case NEQ:
 +                    throw new InvalidRequestException(String.format("Unsupported \"!=\" relation: %s", relation));
              }
          }
  
-         private Restriction getExistingRestriction(SelectStatement stmt, ColumnDefinition def)
+         /**
+          * Checks that the operator for the specified column is compatible with the bounds of the existing restriction.
+          *
+          * @param existing the existing restriction
 -         * @param name the column name
 -         * @param relType the type of relation
++         * @param def the column definition
++         * @param operator the operator
+          * @throws InvalidRequestException if the operator is not compatible with the bounds of the existing restriction
+          */
 -        private static void checkBound(Restriction existing, Name name, Relation.Type relType) throws InvalidRequestException
++        private static void checkBound(Restriction existing, ColumnDefinition def, Operator operator) throws InvalidRequestException
+         {
+             Restriction.Slice existingSlice = (Restriction.Slice) existing;
+ 
 -            if (existingSlice.hasBound(Bound.START) && (relType == Relation.Type.GT || relType == Relation.Type.GTE))
++            if (existingSlice.hasBound(Bound.START) && (operator == Operator.GT || operator == Operator.GTE))
+                 throw new InvalidRequestException(String.format(
 -                            "More than one restriction was found for the start bound on %s", name.name));
++                            "More than one restriction was found for the start bound on %s", def.name));
+ 
 -            if (existingSlice.hasBound(Bound.END) && (relType == Relation.Type.LT || relType == Relation.Type.LTE))
++            if (existingSlice.hasBound(Bound.END) && (operator == Operator.LT || operator == Operator.LTE))
+                 throw new InvalidRequestException(String.format(
 -                            "More than one restriction was found for the end bound on %s", name.name));
++                            "More than one restriction was found for the end bound on %s", def.name));
+         }
+ 
 -        private Restriction getExistingRestriction(SelectStatement stmt, CFDefinition.Name name)
++        private static 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.name);
                  default:
 -                    return stmt.metadataRestrictions.get(name);
 +                    throw new AssertionError();
              }
          }
  
@@@ -1804,45 -1871,23 +1835,47 @@@
                  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));
 +                        else if (existingRestriction.isOnToken() != newRel.onToken)
 +                            // For partition keys, we shouldn't have slice restrictions without token(). And while this is rejected later by
 +                            // processPartitionKeysRestrictions, we shouldn't update the existing restriction by the new one if the old one was using token()
 +                            // and the new one isn't since that would bypass that later test.
 +                            throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
 +
++                        checkBound(existingRestriction, def, newRel.operator());
++
 +                        Term t = newRel.getValue().prepare(keyspace(), receiver);
 +                        t.collectMarkerSpecification(boundNames);
-                         ((SingleColumnRestriction.Slice)existingRestriction).setBound(def.name, newRel.operator(), t);
++                        ((SingleColumnRestriction.Slice)existingRestriction).setBound(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.isOnToken() != newRel.onToken)
 -                        // For partition keys, we shouldn't have slice restrictions without token(). And while this is rejected later by
 -                        // processPartitionKeysRestrictions, we shouldn't update the existing restriction by the new one if the old one was using token()
 -                        // and the new one isn't since that would bypass that later test.
 -                        throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
 -                    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);
 +                        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() == Operator.CONTAINS_KEY;
 +                    receiver = makeCollectionReceiver(receiver, isKey);
 +                    Term t = newRel.getValue().prepare(keyspace(), receiver);
                      t.collectMarkerSpecification(boundNames);
 -                    ((SingleColumnRestriction.Slice)existingRestriction).setBound(newRel.operator(), t);
 +                    ((SingleColumnRestriction.Contains)existingRestriction).add(t, isKey);
 +                    break;
                  }
 -                break;
              }
              return existingRestriction;
          }
@@@ -1958,12 -2003,12 +1991,12 @@@
              // columns must have a EQ, and all following must have no restriction. Unless
              // the column is indexed that is.
              boolean canRestrictFurtherComponents = true;
 -            CFDefinition.Name previous = null;
 +            ColumnDefinition previous = null;
-             boolean previousIsSlice = false;
+             Restriction previousRestriction = null;
 -            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)
@@@ -1972,21 -2017,28 +2005,28 @@@
                  }
                  else if (!canRestrictFurtherComponents)
                  {
--                    // We're here if the previous clustering column was either not restricted or was a slice.
-                     // We can't restrict the current column unless:
-                     //   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.isMultiColumn()))
++                    // We're here if the previous clustering column was either not restricted, was a slice or an IN tulpe-notation.
+ 
+                     // we can continue if we are in the special case of a slice 'tuple' notation from #4851
+                     if (restriction != previousRestriction)
                      {
+                         // if we have a 2ndary index, we need to use it
                          if (hasQueriableIndex)
                          {
-                             stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
+                             stmt.usesSecondaryIndexing = true;
                              break;
                          }
+ 
+                         if (previousRestriction == null)
+                             throw new InvalidRequestException(String.format(
 -                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is not restricted)", cname, previous));
++                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is not restricted)", cdef.name, previous.name));
+ 
+                         if (previousRestriction.isMultiColumn() && previousRestriction.isIN())
+                             throw new InvalidRequestException(String.format(
 -                                 "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by an IN tuple notation)", cname, previous));
++                                     "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by an IN tuple notation)", cdef.name, previous.name));
+ 
                          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)", cdef.name, previous.name));
 -                            "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", cname, previous));
++                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", cdef.name, previous.name));
                      }
                  }
                  else if (restriction.isSlice())
@@@ -2002,18 -2053,16 +2041,23 @@@
                  else if (restriction.isIN())
                  {
                      if (!restriction.isMultiColumn() && i != stmt.columnRestrictions.length - 1)
 -                        throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cname));
 +                        throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cdef.name));
-                     else if (stmt.selectACollection())
++
+                     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("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cdef.name));
+ 
+                     if (restriction.isMultiColumn())
+                         canRestrictFurtherComponents = false;
                  }
 +                else if (restriction.isContains())
 +                {
 +                    if (!hasQueriableIndex)
 +                        throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by a CONTAINS relation without a secondary index", cdef.name));
 +                    stmt.usesSecondaryIndexing = true;
 +                }
  
 -                previous = cname;
 +                previous = cdef;
+                 previousRestriction = restriction;
              }
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9c7a601b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
index b1c6ccc,2e63272..34cd175
--- a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
@@@ -87,9 -78,9 +87,9 @@@ public abstract class SingleColumnRestr
  
      public static class InWithValues extends SingleColumnRestriction implements Restriction.IN
      {
--        protected final List<Term> values;
++        protected final List<? extends Term> values;
  
--        public InWithValues(List<Term> values)
++        public InWithValues(List<? extends Term> values)
          {
              this.values = values;
          }
@@@ -292,7 -253,7 +292,8 @@@
              throw new AssertionError();
          }
  
-         public void setBound(ColumnIdentifier name, Operator operator, Term t) throws InvalidRequestException
 -        public void setBound(Relation.Type type, Term t) throws InvalidRequestException
++        @Override
++        public final void setBound(Operator operator, Term t) throws InvalidRequestException
          {
              Bound b;
              boolean inclusive;
@@@ -318,9 -279,9 +319,7 @@@
                      throw new AssertionError();
              }
  
--            if (bounds[b.idx] != null)
--                throw new InvalidRequestException(String.format(
-                         "More than one restriction was found for the %s bound on %s", b.name().toLowerCase(), name));
 -                        "More than one restriction was found for the %s bound", b.name().toLowerCase()));
++            assert bounds[b.idx] == null;
  
              bounds[b.idx] = t;
              boundInclusive[b.idx] = inclusive;


[2/5] cassandra git commit: Fix mixed single and multi-column relation support

Posted by ty...@apache.org.
Fix mixed single and multi-column relation support

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-8613


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

Branch: refs/heads/cassandra-2.1
Commit: 90a012a1fe29edbb70de30d5f683e10cc27e1602
Parents: fb67c41
Author: blerer <b_...@hotmail.com>
Authored: Thu Mar 5 12:25:03 2015 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Thu Mar 5 12:25:03 2015 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../cql3/statements/SelectStatement.java        | 384 +++++---
 .../cassandra/cql3/MultiColumnRelationTest.java | 186 +++-
 .../cql3/statements/SelectStatementTest.java    | 973 +++++++++++++++++++
 4 files changed, 1386 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/90a012a1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index faa14d5..462a8d1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.0.13:
+ * Fix regression in mixed single and multi-column relation support for
+   SELECT statements (CASSANDRA-8613)
  * Add ability to limit number of native connections (CASSANDRA-8086)
  * Add offline tool to relevel sstables (CASSANDRA-8301)
  * Preserve stream ID for more protocol errors (CASSANDRA-8848)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90a012a1/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 59ed6e1..6b3c781 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
@@ -31,7 +32,6 @@ import org.github.jamm.MemoryMeter;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.CFDefinition.Name;
-import org.apache.cassandra.cql3.CFDefinition.Name.Kind;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
@@ -641,7 +641,13 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 
         // We deal with IN queries for keys in other places, so we know buildBound will return only one result
-        return buildBound(b, cfDef.partitionKeys(), keyRestrictions, false, cfDef.getKeyNameBuilder(), variables).get(0);
+        return buildBound(b,
+                          new ArrayList<Name>(cfDef.partitionKeys()),
+                          keyRestrictions,
+                          false,
+                          cfDef,
+                          cfDef.getKeyNameBuilder(),
+                          variables).get(0);
     }
 
     private Token getTokenBound(Bound b, List<ByteBuffer> variables, IPartitioner<?> p) throws InvalidRequestException
@@ -713,18 +719,35 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
         ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
         Iterator<CFDefinition.Name> idIter = cfDef.clusteringColumns().iterator();
-        for (Restriction r : columnRestrictions)
+        while (idIter.hasNext())
         {
             CFDefinition.Name name = idIter.next();
+            Restriction r = columnRestrictions[name.position];
             assert r != null && !r.isSlice();
 
             if (r.isEQ())
             {
                 List<ByteBuffer> values = r.values(variables);
-                ByteBuffer val = values.get(0);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", name.name));
-                builder.add(val);
+                if (r.isMultiColumn())
+                {
+                    for (int i = 0, m = values.size(); i < m; i++)
+                    {
+                        if (i != 0)
+                            name = idIter.next();
+
+                        ByteBuffer val = values.get(i);
+                        if (val == null)
+                            throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", name));
+                        builder.add(val);
+                    }
+                }
+                else
+                {
+                    ByteBuffer val = values.get(0);
+                    if (val == null)
+                        throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", name.name));
+                    builder.add(val);
+                }
             }
             else
             {
@@ -841,37 +864,23 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return false;
     }
 
-    private List<ByteBuffer> buildBound(Bound bound,
-                                        Collection<CFDefinition.Name> names,
+    @VisibleForTesting
+    static List<ByteBuffer> buildBound(Bound bound,
+                                        List<CFDefinition.Name> names,
                                         Restriction[] restrictions,
                                         boolean isReversed,
+                                        CFDefinition cfDef,
                                         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, (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),
         // it only depends on whether the slice is reversed
         Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-        for (Iterator<CFDefinition.Name> iter = names.iterator(); iter.hasNext();)
+        for (int i = 0; i < names.size(); i++)
         {
-            CFDefinition.Name name = iter.next();
+            CFDefinition.Name name = names.get(i);
 
             // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
             // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
@@ -889,116 +898,130 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             }
             if (r.isSlice())
             {
-                builder.add(getSliceValue(r, b, variables));
+                if (r.isMultiColumn())
+                {
+                    List<ByteBuffer> values = ((MultiColumnRestriction.Slice) r).componentBounds(b, variables);
+                    List<Name> columns = subList(names, i, values.size());
+                    addComponents(builder, columns, values);
+                }
+                else
+                {
+                    addComponent(builder, name, getSliceValue(r, b, variables));
+                }
+
                 Relation.Type relType = ((Restriction.Slice)r).getRelation(eocBound, b);
                 return Collections.singletonList(builder.buildForRelation(relType));
             }
-            else
+
+            if (r.isIN())
             {
+                // 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);
+
+                if (r.isMultiColumn())
+                {
+                    List<List<ByteBuffer>> splitInValues = ((MultiColumnRestriction.IN) r).splitValues(variables);
+                    for (List<ByteBuffer> components : splitInValues)
+                    {
+                        ColumnNameBuilder copy = builder.copy();
+                        List<Name> columns = subList(names, i, components.size());
+                        addComponents(copy, columns, components);
+
+                        inValues.add(buildColumnName(copy, eocBound));
+                    }
+                    return new ArrayList<>(inValues);
+                }
+
                 List<ByteBuffer> values = r.values(variables);
                 if (values.size() != 1)
                 {
                     // 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<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
+
                     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);
-                        // See below for why this
-                        s.add((eocBound == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
+                        ColumnNameBuilder copy = builder.copy();
+                        addComponent(copy, name, val);
+                        inValues.add(buildColumnName(copy, eocBound));
                     }
-                    return new ArrayList<>(s);
+                    return new ArrayList<>(inValues);
                 }
+            }
 
-                ByteBuffer val = values.get(0);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
-                builder.add(val);
+            List<ByteBuffer> values = r.values(variables);
+            if (r.isMultiColumn())
+            {
+                List<Name> columns = subList(names, i, values.size());
+                addComponents(builder, columns, values);
+                i += values.size() - 1;
+            }
+            else
+            {
+                addComponent(builder, name, values.get(0));
             }
         }
-        // Means no relation at all or everything was an equal
+        return Collections.singletonList(buildColumnName(builder, eocBound));
+    }
+
+    /**
+     * Returns a view of the portion of the list starting at the index offset and containing the number of elements
+     * specified.
+     *
+     * @param list the original list
+     * @param offset the index offset
+     * @param length the number of elements
+     * @return a view of the specified range within this list
+     */
+    private static <T> List<T> subList(List<T> list, int offset, int length)
+    {
+        return list.subList(offset, offset + length);
+    }
+
+    /**
+     * Builds the column name when there was not slice.
+     *
+     * @param builder the column name builder
+     * @param eocBound the End of Component bound
+     * @return the column name
+     */
+    private static ByteBuffer buildColumnName(ColumnNameBuilder builder, Bound eocBound)
+    {
         // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
         // 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((eocBound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
+        return (eocBound == 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,
-                                                     MultiColumnRestriction.IN restriction,
-                                                     boolean isReversed,
-                                                     ColumnNameBuilder builder,
-                                                     List<ByteBuffer> variables) throws InvalidRequestException
+    /**
+     * Adds the specified component values to the specified builder.
+     *
+     * @param builder  the ColumnNameBuilder to which the values must be added
+     * @param names the column names
+     * @param values the values to add
+     * @throws InvalidRequestException if one of the values is null
+     */
+    private static void addComponents(ColumnNameBuilder builder, List<Name> names, List<ByteBuffer> values) throws InvalidRequestException
     {
-        List<List<ByteBuffer>> splitInValues = restriction.splitValues(variables);
-        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-
-        // 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);
-        for (List<ByteBuffer> components : splitInValues)
-        {
-            ColumnNameBuilder nameBuilder = builder.copy();
-            for (ByteBuffer component : components)
-                nameBuilder.add(component);
-
-            inValues.add((eocBound == Bound.END && nameBuilder.remainingCount() > 0) ? nameBuilder.buildAsEndOfRange() : nameBuilder.build());
-        }
-        return new ArrayList<>(inValues);
+        for (int i = 0, m = values.size(); i < m; i++)
+            addComponent(builder, names.get(i), values.get(i));
     }
 
-    private List<ByteBuffer> buildMultiColumnEQBound(Bound bound, MultiColumnRestriction.EQ restriction, boolean isReversed, ColumnNameBuilder builder, List<ByteBuffer> variables) throws InvalidRequestException
+    /**
+     * Adds the specified component value to the specified builder
+     *
+     * @param builder the ColumnNameBuilder to which the value must be added
+     * @param name the column associated to the value
+     * @param value the value to add
+     * @throws InvalidRequestException if the value is null
+     */
+    private static void addComponent(ColumnNameBuilder builder, Name name, ByteBuffer value) 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);
+        if (value == null)
+            throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", name));
+        builder.add(value);
     }
 
     private static boolean isNullRestriction(Restriction r, Bound b)
@@ -1010,16 +1033,19 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
     {
         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", r));
-        return val;
+        return slice.bound(b, variables);
     }
 
     private List<ByteBuffer> getRequestedBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
     {
         assert isColumnRange();
-        return buildBound(b, cfDef.clusteringColumns(), columnRestrictions, isReversed, cfDef.getColumnNameBuilder(), variables);
+        return buildBound(b,
+                          new ArrayList<Name>(cfDef.clusteringColumns()),
+                          columnRestrictions,
+                          isReversed,
+                          cfDef,
+                          cfDef.getColumnNameBuilder(),
+                          variables);
     }
 
     public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
@@ -1409,16 +1435,22 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
     {
         Collection<CFDefinition.Name> requestedColumns = selection.getColumns();
         for (CFDefinition.Name name : requestedColumns)
+        {
             if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS && name.kind != CFDefinition.Name.Kind.STATIC)
-                throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", name));
+                throw new InvalidRequestException(String.format(
+                        "SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", name));
+        }
 
         // If it's a key range, we require that all partition key columns are selected so we don't have to bother with post-query grouping.
         if (!isKeyRange)
             return;
 
         for (CFDefinition.Name name : cfDef.partitionKeys())
+        {
             if (!requestedColumns.contains(name))
-                throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
+                throw new InvalidRequestException(String.format(
+                        "SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
+        }
     }
 
     public static class RawStatement extends CFStatement
@@ -1465,8 +1497,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
              */
             boolean hasQueriableIndex = false;
             boolean hasQueriableClusteringColumnIndex = false;
-            boolean hasSingleColumnRelations = false;
-            boolean hasMultiColumnRelations = false;
             for (Relation relation : whereClause)
             {
                 if (relation.isMultiColumn())
@@ -1481,7 +1511,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                         hasQueriableClusteringColumnIndex |= queriable[1];
                         Name name = cfDef.get(entity);
                         names.add(name);
-                        hasMultiColumnRelations |= Kind.COLUMN_ALIAS.equals(name.kind);
                     }
                     updateRestrictionsForRelation(stmt, names, rel, boundNames);
                 }
@@ -1493,12 +1522,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     hasQueriableIndex |= queriable[0];
                     hasQueriableClusteringColumnIndex |= queriable[1];
                     Name name = cfDef.get(entity);
-                    hasSingleColumnRelations |= Kind.COLUMN_ALIAS.equals(name.kind);
                     updateRestrictionsForRelation(stmt, name, rel, boundNames);
                 }
             }
-            if (hasSingleColumnRelations && hasMultiColumnRelations)
-                throw new InvalidRequestException("Mixing single column relations and multi column relations on clustering columns is not allowed");
 
              // At this point, the select statement if fully constructed, but we still have a few things to validate
             processPartitionKeyRestrictions(stmt, cfDef, hasQueriableIndex);
@@ -1592,10 +1618,12 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         {
             List<CFDefinition.Name> restrictedColumns = new ArrayList<>();
             Set<CFDefinition.Name> seen = new HashSet<>();
+            Restriction existing = null;
 
-            int previousPosition = -1;
-            for (CFDefinition.Name name : names)
+            int previousPosition = names.get(0).position - 1;
+            for (int i = 0, m = names.size(); i < m; i++)
             {
+                Name name = names.get(i);
                 // 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));
@@ -1611,20 +1639,51 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                         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(
-                                "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
+
+                    throw new InvalidRequestException(String.format(
+                            "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
                 }
                 previousPosition++;
 
-                Restriction existing = getExistingRestriction(stmt, name);
+                Restriction previous = existing;
+                existing = getExistingRestriction(stmt, name);
                 Relation.Type operator = relation.operator();
                 if (existing != null)
                 {
                     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",
+                                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));
+                    {
+                        throw new InvalidRequestException(String.format(
+                                "Column \"%s\" cannot be restricted by an equality relation and an inequality relation", name));
+                    }
+                    else
+                    {
+                        if (!existing.isMultiColumn())
+                        {
+                            throw new InvalidRequestException(String.format(
+                                    "Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities: %s",
+                                    name, relation));
+                        }
+
+                        boolean existingRestrictionStartBefore =
+                                (i == 0 && name.position != 0 && stmt.columnRestrictions[name.position - 1] == existing);
+
+                        boolean existingRestrictionStartAfter = (i != 0 && previous != existing);
+
+                        if (existingRestrictionStartBefore || existingRestrictionStartAfter)
+                        {
+                            throw new InvalidRequestException(String.format(
+                                    "Column \"%s\" cannot be restricted by two tuple-notation inequalities not starting with the same column: %s",
+                                    name, relation));
+                        }
+
+                        checkBound(existing, name, operator);
+                    }
                 }
                 restrictedColumns.add(name);
             }
@@ -1678,20 +1737,41 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 {
                     Term t = relation.getValue().prepare(names);
                     t.collectMarkerSpecification(boundNames);
+
+                    Restriction.Slice restriction = (Restriction.Slice) getExistingRestriction(stmt, names.get(0));
+                    if (restriction == null)
+                        restriction = new MultiColumnRestriction.Slice(false);
+                    restriction.setBound(relation.operator(), t);
+
                     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: %s", name, relation));
-                        restriction.setBound(relation.operator(), t);
                         stmt.columnRestrictions[name.position] = restriction;
                     }
                 }
             }
         }
 
+        /**
+         * Checks that the operator for the specified column is compatible with the bounds of the existing restriction.
+         *
+         * @param existing the existing restriction
+         * @param name the column name
+         * @param relType the type of relation
+         * @throws InvalidRequestException if the operator is not compatible with the bounds of the existing restriction
+         */
+        private static void checkBound(Restriction existing, Name name, Relation.Type relType) throws InvalidRequestException
+        {
+            Restriction.Slice existingSlice = (Restriction.Slice) existing;
+
+            if (existingSlice.hasBound(Bound.START) && (relType == Relation.Type.GT || relType == Relation.Type.GTE))
+                throw new InvalidRequestException(String.format(
+                            "More than one restriction was found for the start bound on %s", name.name));
+
+            if (existingSlice.hasBound(Bound.END) && (relType == Relation.Type.LT || relType == Relation.Type.LTE))
+                throw new InvalidRequestException(String.format(
+                            "More than one restriction was found for the end bound on %s", name.name));
+        }
+
         private Restriction getExistingRestriction(SelectStatement stmt, CFDefinition.Name name)
         {
             switch (name.kind)
@@ -1924,7 +2004,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             // the column is indexed that is.
             boolean canRestrictFurtherComponents = true;
             CFDefinition.Name previous = null;
-            boolean previousIsSlice = false;
+            Restriction previousRestriction = null;
             Iterator<CFDefinition.Name> iter = cfDef.clusteringColumns().iterator();
             for (int i = 0; i < stmt.columnRestrictions.length; i++)
             {
@@ -1934,29 +2014,35 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 if (restriction == null)
                 {
                     canRestrictFurtherComponents = false;
-                    previousIsSlice = false;
                 }
                 else if (!canRestrictFurtherComponents)
                 {
                     // We're here if the previous clustering column was either not restricted or was a slice.
-                    // We can't restrict the current column unless:
-                    //   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.isMultiColumn()))
+
+                    // we can continue if we are in the special case of a slice 'tuple' notation from #4851
+                    if (restriction != previousRestriction)
                     {
+                        // if we have a 2ndary index, we need to use it
                         if (hasQueriableIndex)
                         {
-                            stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
+                            stmt.usesSecondaryIndexing = true;
                             break;
                         }
+
+                        if (previousRestriction == null)
+                            throw new InvalidRequestException(String.format(
+                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is not restricted)", cname, previous));
+
+                        if (previousRestriction.isMultiColumn() && previousRestriction.isIN())
+                            throw new InvalidRequestException(String.format(
+                                 "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by an IN tuple notation)", 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));
+                            "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", cname, previous));
                     }
                 }
                 else if (restriction.isSlice())
                 {
-                    previousIsSlice = true;
                     canRestrictFurtherComponents = false;
                     Restriction.Slice slice = (Restriction.Slice)restriction;
                     // For non-composite slices, we don't support internally the difference between exclusive and
@@ -1970,9 +2056,13 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                         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));
+
+                    if (restriction.isMultiColumn())
+                        canRestrictFurtherComponents = false;
                 }
 
                 previous = cname;
+                previousRestriction = restriction;
             }
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90a012a1/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
index 30a9226..e3ccba5 100644
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
@@ -65,7 +65,7 @@ public class MultiColumnRelationTest
             executeSchemaChange(
                     "CREATE TABLE IF NOT EXISTS %s.single_partition" + tableSuffix + "(a int PRIMARY KEY, b int)" + compactOption);
             executeSchemaChange(
-                    "CREATE TABLE IF NOT EXISTS %s.compound_partition" +tableSuffix + "(a int, b int, c int, PRIMARY KEY ((a, b)))" + compactOption);
+                    "CREATE TABLE IF NOT EXISTS %s.compound_partition" + tableSuffix + "(a int, b int, c int, PRIMARY KEY ((a, b)))" + compactOption);
             executeSchemaChange(
                     "CREATE TABLE IF NOT EXISTS %s.single_clustering" + tableSuffix + "(a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
             executeSchemaChange(
@@ -135,12 +135,6 @@ public class MultiColumnRelationTest
             return null;
     }
 
-    @Test(expected=InvalidRequestException.class)
-    public void testMixMultiColumnRelationsAndSingleColumn() throws Throwable
-    {
-        execute("SELECT * FROM %s.multiple_clustering WHERE a = 1 AND (b) in ((2),(3)) AND c > 4");
-    }
-
     @Test(expected=SyntaxException.class)
     public void testEmptyIdentifierTuple() throws Throwable
     {
@@ -297,15 +291,25 @@ public class MultiColumnRelationTest
     }
 
     @Test
-    public void testMixSingleAndTupleInequalities() throws Throwable
+    public void testInvalidMultiAndSingleColumnRelationMix() throws Throwable
     {
         for (String tableSuffix : new String[]{"", "_compact"})
         {
             String[] queries = new String[]{
-                    "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > (0, 1, 0) AND b < 1",
-                    "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > (0, 1, 0) AND c < 1",
-                    "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND b > 1 AND (b, c, d) < (1, 1, 0)",
-                    "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND c > 1 AND (b, c, d) < (1, 1, 0)",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c, d) > (0, 1, 0) AND b < 1",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c, d) > (0, 1, 0) AND c < 1",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND b > 1 AND (b, c, d) < (1, 1, 0)",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND c > 1 AND (b, c, d) < (1, 1, 0)",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE (a, b, c, d) IN ((0, 1, 2, 3))",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE (c, d) IN ((0, 1))",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0  AND b > 0  AND (c, d) IN ((0, 0))",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND b > 0  AND (c, d) > (0, 0)",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (c, d) > (0, 0) AND b > 0  ",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) > (0, 0) AND (b) < (0) AND (c) < (0)",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (c) < (0) AND (b, c) > (0, 0) AND (b) < (0)",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b) < (0) AND (c) < (0) AND (b, c) > (0, 0)",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) > (0, 0) AND (c) < (0)",
+                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) in ((0, 0), (0, 0)) AND d > 0"
             };
 
             for (String query : queries)
@@ -323,6 +327,164 @@ public class MultiColumnRelationTest
     }
 
     @Test
+    public void testMultiAndSingleColumnRelationMix() throws Throwable
+    {
+        for (String tableSuffix : new String[]{"", "_compact"})
+        {
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
+
+            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and b = 1 and (c, d) = (0, 0)");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and b = 1 and (c) IN ((0))");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and b = 1 and (c) IN ((0), (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" + tableSuffix
+                    + " WHERE a = 0 and b = 1 and (c, d) IN ((0, 0))");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and b = 1 and (c, d) IN ((0, 0), (1, 1))");
+            assertEquals(2, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+            checkRow(1, results, 0, 1, 1, 1);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and b = 1 and (c, d) > (0, 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" + tableSuffix
+                    + " WHERE a = 0 and b = 1 and (c, d) > (0, 0) and (c) <= (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" + tableSuffix
+                    + " WHERE a = 0 and b = 1 and (c, d) >= (0, 0) and (c, d) < (1, 1)");
+            assertEquals(2, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+            checkRow(1, results, 0, 1, 1, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b, c) = (0, 1) and d = 0");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 0, 1, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and b = 0 and (c) = (1) and d = 0");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 0, 1, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b, c) = (0, 1) and d IN (0, 2)");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 0, 1, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and b = 0 and (c) = (1) and d IN (0, 2)");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 0, 1, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b, c) = (0, 1) and d >= 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" + tableSuffix
+                    + " WHERE a = 0 and d < 1 and (b, c) = (0, 1) and d >= 0");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 0, 1, 0);
+        }
+    }
+
+    @Test
+    public void testMultipleMultiColumnRelation() throws Throwable
+    {
+        for (String tableSuffix : new String[]{"", "_compact"})
+        {
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
+            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
+
+            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c, d) = (0, 0)");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c) = (0) and (d) = (0)");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c) IN ((0))");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c) IN ((0), (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" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c, d) IN ((0, 0))");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c, d) IN ((0, 0), (1, 1))");
+            assertEquals(2, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+            checkRow(1, results, 0, 1, 1, 1);
+
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c, d) > (0, 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" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c, d) > (0, 0) and (c) <= (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" + tableSuffix
+                    + " WHERE a = 0 and (b) = (1) and (c, d) >= (0, 0) and (c, d) < (1, 1)");
+            assertEquals(2, results.size());
+            checkRow(0, results, 0, 1, 0, 0);
+            checkRow(1, results, 0, 1, 1, 0);
+        }
+    }
+
+    @Test
     public void testSingleClusteringColumnInequality() throws Throwable
     {
         for (String tableSuffix : new String[]{"", "_compact"})


[4/5] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/9c7a601b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
index 25df030,e3ccba5..a4ee5d7
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
@@@ -17,65 -17,835 +17,221 @@@
   */
  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
 +public class MultiColumnRelationTest extends CQLTester
  {
 -    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'}");
 -        for (boolean isCompact : new boolean[]{false, true})
 -        {
 -            String tableSuffix = isCompact ? "_compact" : "";
 -            String compactOption = isCompact ? " WITH COMPACT STORAGE" : "";
 -
 -            executeSchemaChange(
 -                    "CREATE TABLE IF NOT EXISTS %s.single_partition" + tableSuffix + "(a int PRIMARY KEY, b int)" + compactOption);
 -            executeSchemaChange(
 -                    "CREATE TABLE IF NOT EXISTS %s.compound_partition" + tableSuffix + "(a int, b int, c int, PRIMARY KEY ((a, b)))" + compactOption);
 -            executeSchemaChange(
 -                    "CREATE TABLE IF NOT EXISTS %s.single_clustering" + tableSuffix + "(a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
 -            executeSchemaChange(
 -                    "CREATE TABLE IF NOT EXISTS %s.multiple_clustering" + tableSuffix + "(a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
 -
 -            compactOption = isCompact ? " COMPACT STORAGE AND " : "";
 -            executeSchemaChange(
 -                    "CREATE TABLE IF NOT EXISTS %s.multiple_clustering_reversed" + tableSuffix +
 -                        "(a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH " + compactOption + " CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
 -        }
 -
 -        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.multiple_clustering_with_indices (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d))");
 -        executeSchemaChange("CREATE INDEX ON %s.multiple_clustering_with_indices (b)");
 -        executeSchemaChange("CREATE INDEX ON %s.multiple_clustering_with_indices (e)");
 -
 -        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.partition_with_indices (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
 -        executeSchemaChange("CREATE INDEX ON %s.partition_with_indices (c)");
 -        executeSchemaChange("CREATE INDEX ON %s.partition_with_indices (f)");
 -
 -        clientState = ClientState.forInternalCalls();
 -    }
 -
 -    @AfterClass
 -    public static void stopGossiper()
 -    {
 -        Gossiper.instance.stop();
 -    }
 -
 -    private static void executeSchemaChange(String query) throws Throwable
 +    @Test
 +    public void testSingleClusteringInvalidQueries() throws Throwable
      {
 -        try
 -        {
 -            process(String.format(query, keyspace), ConsistencyLevel.ONE);
 -        } catch (RuntimeException exc)
 +        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
          {
 -            throw exc.getCause();
 +            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
 +
 +            assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
-             assertInvalid("SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
++            assertInvalidMessage("Column \"b\" cannot be restricted by an equality relation and an inequality relation",
++                                 "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
++            assertInvalidMessage("More than one restriction was found for the start bound on b",
++                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
++            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
++                                 "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
          }
      }
  
 -    private static UntypedResultSet execute(String query) throws Throwable
 +    @Test
 +    public void testMultiClusteringInvalidQueries() throws Throwable
      {
 -        try
 -        {
 -            return processInternal(String.format(query, keyspace));
 -        } catch (RuntimeException exc)
 +        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
          {
 -            if (exc.getCause() != null)
 -                throw exc.getCause();
 -            throw exc;
 +            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
 +
 +            assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()");
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
++            assertInvalidMessage("Expected 2 elements in value tuple, but got 3: (?, ?, ?)",
++                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
++            assertInvalidMessage("Invalid null value in condition for column c",
++                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
 +
 +            // Wrong order of columns
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
++            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) = (?, ?, ?)",
++                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
++            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) > (?, ?, ?)",
++                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
 +
 +            // Wrong number of values
-             assertInvalid("SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
-             assertInvalid("SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
++            assertInvalidMessage("Expected 3 elements in value tuple, but got 2: (?, ?)",
++                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
++            assertInvalidMessage("Expected 3 elements in value tuple, but got 5: (?, ?, ?, ?, ?)",
++                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
 +
 +            // Missing first clustering column
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
++                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
++                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
 +
 +            // Nulls
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
++            assertInvalidMessage("Invalid null value in condition for column d",
++                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
 +
 +            // Wrong type for 'd'
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
++            assertInvalidMessage("Expected 4 or 0 byte int (6)",
++                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
 +
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
++            assertInvalidMessage("Invalid tuple type literal for b of type int",
++                                 "SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
 +
 +            // Mix single and tuple inequalities
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-             assertInvalid("SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
++            assertInvalidMessage("Column \"b\" cannot be restricted by both a tuple notation inequality and a single column inequality (b < ?)",
++                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
++            assertInvalidMessage("Column \"c\" cannot be restricted by both a tuple notation inequality and a single column inequality (c < ?)",
++                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
++            assertInvalidMessage("Column \"b\" cannot have both tuple-notation inequalities and single-column inequalities: (b, c, d) < (?, ?, ?)",
++                                 "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
++            assertInvalidMessage("Column \"c\" cannot have both tuple-notation inequalities and single-column inequalities: (b, c, d) < (?, ?, ?)",
++                                 "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
++
++            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
++                                 "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
++                                 "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
++                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
++
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
++                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
++                                 "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
++            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (c) < (?)",
++                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
++            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (b, c) > (?, ?)",
++                                 "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
++            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (b, c) > (?, ?)",
++                                 "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
++
++            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (c) < (?)",
++                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
++
++            assertInvalidMessage("PRIMARY KEY column \"d\" cannot be restricted (preceding column \"c\" is restricted by an IN tuple notation)",
++                                 "SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
+         }
+     }
+ 
 -    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
++    public void testMultiAndSingleColumnRelationMix() throws Throwable
+     {
 -        UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((1, 2, null))");
 -        assertTrue(results.isEmpty());
 -    }
++        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
++        {
++            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+ 
 -    @Test(expected=InvalidRequestException.class)
 -    public void testPartitionKeyInequality() throws Throwable
 -    {
 -        execute("SELECT * FROM %s.single_partition WHERE (a) > (1)");
 -    }
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+ 
 -    @Test(expected=InvalidRequestException.class)
 -    public void testPartitionKeyEquality() throws Throwable
 -    {
 -        execute("SELECT * FROM %s.single_partition WHERE (a) = (0)");
 -    }
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+ 
 -    @Test(expected=InvalidRequestException.class)
 -    public void testRestrictNonPrimaryKey() throws Throwable
 -    {
 -        execute("SELECT * FROM %s.single_partition WHERE (b) = (0)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
++                       row(0, 1, 0, 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)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
++                       row(0, 1, 0, 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)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?), (?))", 0, 1, 0, 1),
++                       row(0, 1, 0, 0),
++                       row(0, 1, 1, 0),
++                       row(0, 1, 1, 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)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?))", 0, 1, 0, 0),
++                       row(0, 1, 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)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
++                       row(0, 1, 0, 0),
++                       row(0, 1, 1, 1));
+ 
 -    @Test(expected=InvalidRequestException.class)
 -    public void testSkipClusteringColumnInInequality() throws Throwable
 -    {
 -        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) > (0, 0)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?)", 0, 1, 0, 0),
++                       row(0, 1, 1, 0),
++                       row(0, 1, 1, 1));
+ 
 -    @Test
 -    public void testSingleClusteringColumnEquality() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            execute("INSERT INTO %s.single_clustering" + tableSuffix + "(a, b, c) VALUES (0, 0, 0)");
 -            execute("INSERT INTO %s.single_clustering" + tableSuffix + " (a, b, c) VALUES (0, 1, 0)");
 -            execute("INSERT INTO %s.single_clustering" + tableSuffix + " (a, b, c) VALUES (0, 2, 0)");
 -            UntypedResultSet results = execute("SELECT * FROM %s.single_clustering" + tableSuffix + " WHERE a=0 AND (b) = (1)");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.single_clustering" + tableSuffix + " WHERE a=0 AND (b) = (3)");
 -            assertEquals(0, results.size());
 -        }
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
++                       row(0, 1, 1, 0),
++                       row(0, 1, 1, 1));
+ 
 -    @Test
 -    public void testMultipleClusteringColumnEquality() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 2, 0, 0)");
 -            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " WHERE a=0 AND b=2 and c=0 and d=0");
 -        }
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
++                       row(0, 1, 0, 0),
++                       row(0, 1, 1, 0));
+ 
 -    @Test(expected=InvalidRequestException.class)
 -    public void testPartitionAndClusteringColumnEquality() throws Throwable
 -    {
 -        execute("SELECT * FROM %s.single_clustering WHERE (a, b) = (0, 0)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
++                       row(0, 0, 1, 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)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
++                       row(0, 0, 1, 0));
+ 
 -    @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')");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d IN (?, ?)", 0, 0, 1, 0, 2),
++                       row(0, 0, 1, 0));
+ 
 -    @Test(expected=SyntaxException.class)
 -    public void testSingleColumnTupleRelation() throws Throwable
 -    {
 -        execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND b = (1, 2, 3)");
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d IN (?, ?)", 0, 0, 1, 0, 2),
++                       row(0, 0, 1, 0));
+ 
 -    @Test
 -    public void testInvalidMultiAndSingleColumnRelationMix() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            String[] queries = new String[]{
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c, d) > (0, 1, 0) AND b < 1",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c, d) > (0, 1, 0) AND c < 1",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND b > 1 AND (b, c, d) < (1, 1, 0)",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND c > 1 AND (b, c, d) < (1, 1, 0)",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE (a, b, c, d) IN ((0, 1, 2, 3))",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE (c, d) IN ((0, 1))",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0  AND b > 0  AND (c, d) IN ((0, 0))",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND b > 0  AND (c, d) > (0, 0)",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (c, d) > (0, 0) AND b > 0  ",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) > (0, 0) AND (b) < (0) AND (c) < (0)",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (c) < (0) AND (b, c) > (0, 0) AND (b) < (0)",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b) < (0) AND (c) < (0) AND (b, c) > (0, 0)",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) > (0, 0) AND (c) < (0)",
 -                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) in ((0, 0), (0, 0)) AND d > 0"
 -            };
 -
 -            for (String query : queries)
 -            {
 -                try
 -                {
 -                    execute(query);
 -                    fail(String.format("Expected query \"%s\" to throw an InvalidRequestException", query));
 -                }
 -                catch (InvalidRequestException e)
 -                {
 -                }
 -            }
 -        }
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
++                       row(0, 0, 1, 0),
++                       row(0, 0, 1, 1));
+ 
 -    @Test
 -    public void testMultiAndSingleColumnRelationMix() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
 -
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
 -
 -            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and b = 1 and (c, d) = (0, 0)");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and b = 1 and (c) IN ((0))");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and b = 1 and (c) IN ((0), (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" + tableSuffix
 -                    + " WHERE a = 0 and b = 1 and (c, d) IN ((0, 0))");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and b = 1 and (c, d) IN ((0, 0), (1, 1))");
 -            assertEquals(2, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -            checkRow(1, results, 0, 1, 1, 1);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and b = 1 and (c, d) > (0, 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" + tableSuffix
 -                    + " WHERE a = 0 and b = 1 and (c, d) > (0, 0) and (c) <= (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" + tableSuffix
 -                    + " WHERE a = 0 and b = 1 and (c, d) >= (0, 0) and (c, d) < (1, 1)");
 -            assertEquals(2, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -            checkRow(1, results, 0, 1, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b, c) = (0, 1) and d = 0");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and b = 0 and (c) = (1) and d = 0");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b, c) = (0, 1) and d IN (0, 2)");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and b = 0 and (c) = (1) and d IN (0, 2)");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b, c) = (0, 1) and d >= 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" + tableSuffix
 -                    + " WHERE a = 0 and d < 1 and (b, c) = (0, 1) and d >= 0");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 1, 0);
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
++                       row(0, 0, 1, 0));
+         }
+     }
+ 
+     @Test
+     public void testMultipleMultiColumnRelation() throws Throwable
+     {
 -        for (String tableSuffix : new String[]{"", "_compact"})
++        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+         {
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
 -
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
 -
 -            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c, d) = (0, 0)");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c) = (0) and (d) = (0)");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c) IN ((0))");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c) IN ((0), (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" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c, d) IN ((0, 0))");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c, d) IN ((0, 0), (1, 1))");
 -            assertEquals(2, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -            checkRow(1, results, 0, 1, 1, 1);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c, d) > (0, 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" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c, d) > (0, 0) and (c) <= (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" + tableSuffix
 -                    + " WHERE a = 0 and (b) = (1) and (c, d) >= (0, 0) and (c, d) < (1, 1)");
 -            assertEquals(2, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -            checkRow(1, results, 0, 1, 1, 0);
 -        }
 -    }
++            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+ 
 -    @Test
 -    public void testSingleClusteringColumnInequality() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            execute("INSERT INTO %s.single_clustering" + tableSuffix + " (a, b, c) VALUES (0, 0, 0)");
 -            execute("INSERT INTO %s.single_clustering" + tableSuffix + " (a, b, c) VALUES (0, 1, 0)");
 -            execute("INSERT INTO %s.single_clustering" + tableSuffix + " (a, b, c) VALUES (0, 2, 0)");
 -
 -            UntypedResultSet results = execute("SELECT * FROM %s.single_clustering" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " WHERE a=0 AND (b) > (0) AND (b) < (2)");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 1, 0);
 -        }
 -    }
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+ 
 -    @Test
 -    public void testMultipleClusteringColumnInequality() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
 -
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
 -
 -            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
++            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+ 
 -            // reversed
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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);
 -        }
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
++                       row(0, 1, 0, 0));
+ 
 -    @Test
 -    public void testMultipleClusteringColumnInequalityReversedComponents() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            // b and d are reversed in the clustering order
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
 -
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
 -
 -
 -            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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);
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) = (?) and (d) = (?)", 0, 1, 0, 0),
++                       row(0, 1, 0, 0));
+ 
 -            // preserve pre-6875 behavior (even though the query result is technically incorrect)
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 AND (b, c) > (1, 0)");
 -            assertEquals(0, results.size());
 -        }
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
++                       row(0, 1, 0, 0));
+ 
 -    @Test
 -    public void testLiteralIn() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?), (?))", 0, 1, 0, 1),
++                       row(0, 1, 0, 0),
++                       row(0, 1, 1, 0),
++                       row(0, 1, 1, 1));
+ 
 -            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix + " 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);
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?))", 0, 1, 0, 0),
++                       row(0, 1, 0, 0));
+ 
 -            // same query, but reversed order for the IN values
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix + " 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" + tableSuffix + " 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" + tableSuffix + " 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);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c) IN ((0, 1)) ORDER BY b DESC, c DESC, d DESC");
 -            assertEquals(2, results.size());
 -            checkRow(0, results, 0, 0, 1, 1);
 -            checkRow(1, results, 0, 0, 1, 0);
 -        }
 -    }
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
++                       row(0, 1, 0, 0),
++                       row(0, 1, 1, 1));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?)", 0, 1, 0, 0),
++                       row(0, 1, 1, 0),
++                       row(0, 1, 1, 1));
+ 
 -    @Test
 -    public void testLiteralInReversed() throws Throwable
 -    {
 -        for (String tableSuffix : new String[]{"", "_compact"})
 -        {
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
 -            execute("INSERT INTO %s.multiple_clustering_reversed" + tableSuffix + " (a, b, c, d) VALUES (0, -1, 0, 0)");
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
++                       row(0, 1, 1, 0),
++                       row(0, 1, 1, 1));
  
-             assertInvalid("SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
-             assertInvalid("SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
 -            UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 AND (b, c, d) IN ((0, 1, 0), (0, 1, 1))");
 -            assertEquals(2, results.size());
 -            checkRow(0, results, 0, 0, 1, 1);
 -            checkRow(1, results, 0, 0, 1, 0);
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
++                       row(0, 1, 0, 0),
++                       row(0, 1, 1, 0));
  
-             assertInvalid("SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
 -            // same query, but reversed order for the IN values
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 AND (b, c, d) IN ((0, 1, 1), (0, 1, 0))");
 -            assertEquals(2, results.size());
 -            checkRow(0, results, 0, 0, 1, 1);
 -            checkRow(1, results, 0, 0, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 AND (b, c, d) IN ((1, 0, 0), (0, 0, 0), (0, 1, 1), (0, 1, 0), (-1, 0, 0))");
 -            assertEquals(5, results.size());
 -            checkRow(0, results, 0, 1, 0, 0);
 -            checkRow(1, results, 0, 0, 0, 0);
 -            checkRow(2, results, 0, 0, 1, 1);
 -            checkRow(3, results, 0, 0, 1, 0);
 -            checkRow(4, results, 0, -1, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 AND (b, c, d) IN ((0, 0, 0))");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 AND (b, c, d) IN ((0, 1, 1))");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 1, 1);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 AND (b, c, d) IN ((0, 1, 0))");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 and (b, c) IN ((0, 1))");
 -            assertEquals(2, results.size());
 -            checkRow(0, results, 0, 0, 1, 1);
 -            checkRow(1, results, 0, 0, 1, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 and (b, c) IN ((0, 0))");
 -            assertEquals(1, results.size());
 -            checkRow(0, results, 0, 0, 0, 0);
 -
 -            results = execute("SELECT * FROM %s.multiple_clustering_reversed" + tableSuffix + " WHERE a=0 and (b) IN ((0))");
 -            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);
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and (d) = (?)", 0, 0, 1, 0),
++                       row(0, 0, 1, 0));
          }
      }