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:35:08 UTC

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

Repository: cassandra
Updated Branches:
  refs/heads/trunk 258e59fcf -> 71778eec2


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;
+    }
+}


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

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
index 51625da,a4ee5d7..8bd4e97
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
@@@ -64,39 -64,174 +64,225 @@@ public class MultiColumnRelationTest ex
                                   "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
  
              // Missing first clustering column
-             assertInvalidMessage("Clustering columns may not be skipped in multi-column relations. They should appear in the PRIMARY KEY order. Got (c, d) = (?, ?)",
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
                                   "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
-             assertInvalidMessage("Clustering columns may not be skipped in multi-column relations. They should appear in the PRIMARY KEY order. Got (c, d) > (?, ?)",
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
                                   "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
  
              // Nulls
--            assertInvalidMessage("Invalid null value in condition for column d",
++            assertInvalidMessage("Invalid null value in condition for columns: [b, c, d]",
                                   "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
  
              // Wrong type for 'd'
 -            assertInvalidMessage("Expected 4 or 0 byte int (6)",
 -                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
 -
 -            assertInvalidMessage("Invalid tuple type literal for b of type int",
 -                                 "SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
 +            assertInvalid("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);
  
              // Mix single and tuple inequalities
 -            assertInvalidMessage("Column \"b\" cannot be restricted by both a tuple notation inequality and a single column inequality (b < ?)",
 +            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
                                   "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 < ?)",
 +            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
                                   "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) < (?, ?, ?)",
 +            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
                                   "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) < (?, ?, ?)",
 +            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
                                   "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",
 +            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
                                   "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
-             assertInvalidMessage("Clustering columns may not be skipped in multi-column relations. They should appear in the PRIMARY KEY order. Got (c, d) IN ((?, ?))",
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
++            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as 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)",
 +
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                  "SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?",
-                                  0, 0, 0, 0, 0, 0);
++            assertInvalidMessage("Clustering 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)",
++            assertInvalidMessage("Clustering 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) < (?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
+                                  "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) > (?, ?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
+                                  "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) > (?, ?)",
++            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                  "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) < (?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
+                                  "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);
+         }
+     }
+ 
+     @Test
+     public void testMultiAndSingleColumnRelationMix() throws Throwable
+     {
+         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);
+ 
+             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);
+ 
+             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);
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
+                        row(0, 1, 0, 0));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
++                       row(0, 0, 0, 0),
++                       row(0, 1, 0, 0));
++
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
+                        row(0, 1, 0, 0));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c) IN ((?))", 0, 0, 1, 0),
++                       row(0, 0, 0, 0),
++                       row(0, 1, 0, 0));
++
+             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));
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?))", 0, 1, 0, 0),
+                        row(0, 1, 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));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
++                       row(0, 0, 0, 0),
++                       row(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));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
++                       row(0, 0, 1, 0),
++                       row(0, 0, 1, 1),
++                       row(0, 1, 1, 0),
++                       row(0, 1, 1, 1));
++
+             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));
+ 
+             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));
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
+                        row(0, 0, 1, 0));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
++                       row(0, 0, 0, 0),
++                       row(0, 0, 1, 0));
++
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
+                        row(0, 0, 1, 0));
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d IN (?, ?)", 0, 0, 1, 0, 2),
+                        row(0, 0, 1, 0));
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d IN (?, ?)", 0, 0, 1, 0, 2),
+                        row(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),
+                        row(0, 0, 1, 1));
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
+                        row(0, 0, 1, 0));
++
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) IN ((?, ?), (?, ?)) and d >= ?", 0, 0, 1, 0, 0, 0),
++                       row(0, 0, 0, 0),
++                       row(0, 0, 1, 0));
+         }
+     }
+ 
+     @Test
 -    public void testMultipleMultiColumnRelation() throws Throwable
++    public void testSeveralMultiColumnRelation() throws Throwable
+     {
+         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);
+ 
+             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);
+ 
+             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);
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
+                        row(0, 1, 0, 0));
+ 
 -            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) = (?) and (d) = (?)", 0, 1, 0, 0),
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
++                       row(0, 0, 0, 0),
+                        row(0, 1, 0, 0));
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
+                        row(0, 1, 0, 0));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c) IN ((?))", 0, 0, 1, 0),
++                       row(0, 0, 0, 0),
++                       row(0, 1, 0, 0));
++
+             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));
+ 
+             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?))", 0, 1, 0, 0),
+                        row(0, 1, 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));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
++                       row(0, 0, 0, 0),
++                       row(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));
+ 
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
++                       row(0, 0, 1, 0),
++                       row(0, 0, 1, 1),
++                       row(0, 1, 1, 0),
++                       row(0, 1, 1, 1));
++
+             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));
+ 
+             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));
+ 
 -            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and (d) = (?)", 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));
++
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
++                       row(0, 0, 0, 0),
++                       row(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));
 +
-             assertInvalid("SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
++            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) IN ((?, ?), (?, ?)) and (d) >= (?)", 0, 0, 1, 0, 0, 0),
++                       row(0, 0, 0, 0),
+                        row(0, 0, 1, 0));
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/test/unit/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSetTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSetTest.java
index 0000000,0000000..05d6e98
new file mode 100644
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSetTest.java
@@@ -1,0 -1,0 +1,1120 @@@
++/*
++ * 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.restrictions;
++
++import java.nio.ByteBuffer;
++import java.util.ArrayList;
++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.Term.MultiItemTerminal;
++import org.apache.cassandra.cql3.statements.Bound;
++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.Int32Type;
++import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.utils.ByteBufferUtil;
++
++import static java.util.Arrays.asList;
++import static org.junit.Assert.assertEquals;
++import static org.junit.Assert.assertTrue;
++
++public class PrimaryKeyRestrictionSetTest
++{
++    @Test
++    public void testBoundsAsCompositesWithNoRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(1);
++
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++    }
++
++    /**
++     * Test 'clustering_0 = 1' with only one clustering column
++     */
++    @Test
++    public void testBoundsAsCompositesWithOneEqRestrictionsAndOneClusteringColumn() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(1);
++
++        ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
++        Restriction eq = newSingleEq(cfMetaData, 0, clustering_0);
++
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), clustering_0, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), clustering_0, EOC.END);
++    }
++
++    /**
++     * Test 'clustering_1 = 1' with 2 clustering columns
++     */
++    @Test
++    public void testBoundsAsCompositesWithOneEqRestrictionsAndTwoClusteringColumns() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(2);
++
++        ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
++        Restriction eq = newSingleEq(cfMetaData, 0, clustering_0);
++
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), clustering_0, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), clustering_0, EOC.END);
++    }
++
++    /**
++     * Test 'clustering_0 IN (1, 2, 3)' with only one clustering column
++     */
++    @Test
++    public void testBoundsAsCompositesWithOneInRestrictionsAndOneClusteringColumn() throws InvalidRequestException
++    {
++        ByteBuffer value1 = ByteBufferUtil.bytes(1);
++        ByteBuffer value2 = ByteBufferUtil.bytes(2);
++        ByteBuffer value3 = ByteBufferUtil.bytes(3);
++
++        CFMetaData cfMetaData = newCFMetaData(2);
++
++        Restriction in = newSingleIN(cfMetaData, 0, value1, value2, value3);
++
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(in);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(3, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.START);
++        assertComposite(bounds.get(1), value2, EOC.START);
++        assertComposite(bounds.get(2), value3, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(3, bounds.size());
++        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 testBoundsAsCompositesWithSliceRestrictionsAndOneClusteringColumn() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(1);
++
++        ByteBuffer value1 = ByteBufferUtil.bytes(1);
++        ByteBuffer value2 = ByteBufferUtil.bytes(2);
++
++        Restriction slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        slice = newSingleSlice(cfMetaData, 0, Bound.END, true, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.END);
++
++        slice = newSingleSlice(cfMetaData, 0, Bound.END, false, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.START);
++
++        slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
++        Restriction slice2 = newSingleSlice(cfMetaData, 0, Bound.END, false, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value2, EOC.START);
++
++        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
++        slice2 = newSingleSlice(cfMetaData, 0, Bound.END, true, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value2, EOC.END);
++    }
++
++    /**
++     * Test 'clustering_0 = 1 AND clustering_1 IN (1, 2, 3)'
++     */
++    @Test
++    public void testBoundsAsCompositesWithEqAndInRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(2);
++
++        ByteBuffer value1 = ByteBufferUtil.bytes(1);
++        ByteBuffer value2 = ByteBufferUtil.bytes(2);
++        ByteBuffer value3 = ByteBufferUtil.bytes(3);
++        Restriction eq = newSingleEq(cfMetaData, 0, value1);
++        Restriction in = newSingleIN(cfMetaData, 1, value1, value2, value3);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq).mergeWith(in);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(3, bounds.size());
++        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 = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(3, bounds.size());
++        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 equal and slice restrictions (e.g 'clustering_0 = 0 clustering_1 > 1')
++     */
++    @Test
++    public void testBoundsAsCompositesWithEqAndSliceRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(2);
++
++        ByteBuffer value1 = ByteBufferUtil.bytes(1);
++        ByteBuffer value2 = ByteBufferUtil.bytes(2);
++        ByteBuffer value3 = ByteBufferUtil.bytes(3);
++
++        Restriction eq = newSingleEq(cfMetaData, 0, value3);
++
++        Restriction slice = newSingleSlice(cfMetaData, 1, Bound.START, false, value1);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq).mergeWith(slice);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, value1, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, EOC.END);
++
++        slice = newSingleSlice(cfMetaData, 1, Bound.START, true, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq).mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, value1, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, EOC.END);
++
++        slice = newSingleSlice(cfMetaData, 1, Bound.END, true, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq).mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, value1, EOC.END);
++
++        slice = newSingleSlice(cfMetaData, 1, Bound.END, false, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq).mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, value1, EOC.START);
++
++        slice = newSingleSlice(cfMetaData, 1, Bound.START, false, value1);
++        Restriction slice2 = newSingleSlice(cfMetaData, 1, Bound.END, false, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq).mergeWith(slice).mergeWith(slice2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, value1, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, value2, EOC.START);
++
++        slice = newSingleSlice(cfMetaData, 1, Bound.START, true, value1);
++        slice2 = newSingleSlice(cfMetaData, 1, Bound.END, true, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq).mergeWith(slice).mergeWith(slice2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, value1, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value3, value2, EOC.END);
++    }
++
++    /**
++     * Test '(clustering_0, clustering_1) = (1, 2)' with two clustering column
++     */
++    @Test
++    public void testBoundsAsCompositesWithMultiEqRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(2);
++
++        ByteBuffer value1 = ByteBufferUtil.bytes(1);
++        ByteBuffer value2 = ByteBufferUtil.bytes(2);
++        Restriction eq = newMultiEq(cfMetaData, 0, value1, value2);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(eq);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        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 testBoundsAsCompositesWithMultiInRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(2);
++
++        ByteBuffer value1 = ByteBufferUtil.bytes(1);
++        ByteBuffer value2 = ByteBufferUtil.bytes(2);
++        ByteBuffer value3 = ByteBufferUtil.bytes(3);
++        Restriction in = newMultiIN(cfMetaData, 0, asList(value1, value2), asList(value2, value3));
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(in);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(2, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, EOC.START);
++        assertComposite(bounds.get(1), value2, value3, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(2, bounds.size());
++        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 testBoundsAsCompositesWithMultiSliceRestrictionsWithOneClusteringColumn() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(1);
++
++        ByteBuffer value1 = ByteBufferUtil.bytes(1);
++        ByteBuffer value2 = ByteBufferUtil.bytes(2);
++
++        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.END);
++
++        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.START);
++
++        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
++        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value2, EOC.START);
++
++        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
++        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value2, EOC.END);
++    }
++
++    /**
++     * Test multi-column slice restrictions (e.g '(clustering_0, clustering_1) > (1, 2)')
++     */
++    @Test
++    public void testBoundsAsCompositesWithMultiSliceRestrictionsWithTwoClusteringColumn() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(2);
++
++        ByteBuffer value1 = ByteBufferUtil.bytes(1);
++        ByteBuffer value2 = ByteBufferUtil.bytes(2);
++
++        // (clustering_0, clustering1) > (1, 2)
++        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        // (clustering_0, clustering1) >= (1, 2)
++        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        // (clustering_0, clustering1) <= (1, 2)
++        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, EOC.END);
++
++        // (clustering_0, clustering1) < (1, 2)
++        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertEmptyComposite(bounds.get(0));
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, EOC.START);
++
++        // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
++        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
++        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value2, EOC.START);
++
++        // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
++        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
++        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        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 testBoundsAsCompositesWithSingleEqAndMultiEqRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(4);
++
++        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)
++        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
++        Restriction multiEq = newMultiEq(cfMetaData, 1, value2, value3);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
++
++        // clustering_0 = 1 AND clustering_1 = 2 AND (clustering_2, clustering_3) = (3, 4)
++        singleEq = newSingleEq(cfMetaData, 0, value1);
++        Restriction singleEq2 = newSingleEq(cfMetaData, 1, value2);
++        multiEq = newMultiEq(cfMetaData, 2, value3, value4);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(singleEq).mergeWith(singleEq2).mergeWith(multiEq);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
++
++        // (clustering_0, clustering_1) = (1, 2) AND clustering_2 = 3
++        singleEq = newSingleEq(cfMetaData, 2, value3);
++        multiEq = newMultiEq(cfMetaData, 0, value1, value2);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
++
++        // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3) AND clustering_3 = 4
++        singleEq = newSingleEq(cfMetaData, 0, value1);
++        singleEq2 = newSingleEq(cfMetaData, 3, value4);
++        multiEq = newMultiEq(cfMetaData, 1, value2, value3);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq).mergeWith(singleEq2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        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 testBoundsAsCompositesWithSingleEqAndMultiINRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(4);
++
++        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))
++        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
++        Restriction multiIN = newMultiIN(cfMetaData, 1, asList(value2, value3), asList(value4, value5));
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiIN);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(2, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
++        assertComposite(bounds.get(1), value1, value4, value5, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(2, bounds.size());
++        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 = newSingleEq(cfMetaData, 0, value1);
++        multiIN = newMultiIN(cfMetaData, 1, asList(value2, value3));
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(multiIN).mergeWith(singleEq);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
++
++        // clustering_0 = 1 AND clustering_1 = 5 AND (clustering_2, clustering_3) IN ((2, 3), (4, 5))
++        singleEq = newSingleEq(cfMetaData, 0, value1);
++        Restriction singleEq2 = newSingleEq(cfMetaData, 1, value5);
++        multiIN = newMultiIN(cfMetaData, 2, asList(value2, value3), asList(value4, value5));
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiIN).mergeWith(singleEq2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(2, bounds.size());
++        assertComposite(bounds.get(0), value1, value5, value2, value3, EOC.START);
++        assertComposite(bounds.get(1), value1, value5, value4, value5, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(2, bounds.size());
++        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 testBoundsAsCompositesWithSingleEqAndSliceRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(3);
++
++        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)
++        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
++        Restriction multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiSlice);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, EOC.END);
++
++        // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3) AND (clustering_1) < (4)
++        singleEq = newSingleEq(cfMetaData, 0, value1);
++        multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3);
++        Restriction multiSlice2 = newMultiSlice(cfMetaData, 1, Bound.END, false, value4);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(multiSlice2).mergeWith(singleEq).mergeWith(multiSlice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        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 = newSingleEq(cfMetaData, 0, value1);
++        multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, true, value2, value3);
++        multiSlice2 = newMultiSlice(cfMetaData, 1, Bound.END, true, value4, value5);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(multiSlice2).mergeWith(singleEq).mergeWith(multiSlice);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.NONE);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        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 testBoundsAsCompositesWithMultiEqAndSingleSliceRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(3);
++
++        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
++        Restriction multiEq = newMultiEq(cfMetaData, 0, value1, value2);
++        Restriction singleSlice = newSingleSlice(cfMetaData, 2, Bound.START, false, value3);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(multiEq).mergeWith(singleSlice);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0),  value1, value2, EOC.END);
++    }
++
++    @Test
++    public void testBoundsAsCompositesWithSeveralMultiColumnRestrictions() throws InvalidRequestException
++    {
++        CFMetaData cfMetaData = newCFMetaData(4);
++
++        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)
++        Restriction multiEq = newMultiEq(cfMetaData, 0, value1, value2);
++        Restriction multiSlice = newMultiSlice(cfMetaData, 2, Bound.START, false, value3, value4);
++        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(multiEq).mergeWith(multiSlice);
++
++        List<Composite> bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        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 = newMultiEq(cfMetaData, 0, value1, value2);
++        Restriction multiIN = newMultiIN(cfMetaData, 2, asList(value3, value4), asList(value4, value5));
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(multiEq).mergeWith(multiIN);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(2, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
++        assertComposite(bounds.get(1), value1, value2, value4, value5, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(2, bounds.size());
++        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)
++        multiEq = newMultiEq(cfMetaData, 0, value1, value2);
++        Restriction multiEq2 = newMultiEq(cfMetaData, 2, value3, value4);
++        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
++        restrictions = restrictions.mergeWith(multiEq).mergeWith(multiEq2);
++
++        bounds = restrictions.boundsAsComposites(Bound.START, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
++
++        bounds = restrictions.boundsAsComposites(Bound.END, QueryOptions.DEFAULT);
++        assertEquals(1, bounds.size());
++        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
++    }
++
++    /**
++     * Asserts that the specified <code>Composite</code> is an empty one.
++     *
++     * @param composite the composite to check
++     */
++    private static void assertEmptyComposite(Composite composite)
++    {
++        assertEquals(Composites.EMPTY, composite);
++    }
++
++    /**
++     * Asserts that the specified <code>Composite</code> contains the specified element and the specified EOC.
++     *
++     * @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 assertComposite(Composite composite, ByteBuffer element, EOC eoc)
++    {
++        assertComposite(composite, eoc, element);
++    }
++
++    /**
++     * Asserts that the specified <code>Composite</code> contains the 2 specified element and the specified EOC.
++     *
++     * @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 assertComposite(Composite composite, ByteBuffer firstElement, ByteBuffer secondElement, EOC eoc)
++    {
++        assertComposite(composite, eoc, firstElement, secondElement);
++    }
++
++    /**
++     * Asserts that the specified <code>Composite</code> contains the 3 specified element and the specified EOC.
++     *
++     * @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 void assertComposite(Composite composite,
++                                        ByteBuffer firstElement,
++                                        ByteBuffer secondElement,
++                                        ByteBuffer thirdElement,
++                                        EOC eoc)
++    {
++        assertComposite(composite, eoc, firstElement, secondElement, thirdElement);
++    }
++
++    /**
++     * Asserts that the specified <code>Composite</code> contains the 4 specified element and the specified EOC.
++     *
++     * @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 void assertComposite(Composite composite,
++                                        ByteBuffer firstElement,
++                                        ByteBuffer secondElement,
++                                        ByteBuffer thirdElement,
++                                        ByteBuffer fourthElement,
++                                        EOC eoc)
++    {
++        assertComposite(composite, eoc, firstElement, secondElement, thirdElement, fourthElement);
++    }
++
++    /**
++     * Asserts that the specified <code>Composite</code> contains the specified elements and EOC.
++     *
++     * @param composite the composite to check
++     * @param eoc the expected EOC of the composite
++     * @param elements the expected elements of the composite
++     */
++    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];
++            assertTrue(String.format("the element %s of the composite is not the expected one: expected %s but was %s",
++                                     i,
++                                     ByteBufferUtil.toInt(element),
++                                     ByteBufferUtil.toInt(composite.get(i))),
++                       element.equals(composite.get(i)));
++        }
++        assertEquals("the EOC of the composite is not the expected one:", eoc, composite.eoc());
++    }
++
++    /**
++     * Creates a new <code>CFMetaData</code> instance.
++     *
++     * @param numberOfClusteringColumns the number of clustering column
++     * @return a new <code>CFMetaData</code> instance
++     */
++    private static CFMetaData newCFMetaData(int numberOfClusteringColumns)
++    {
++        List<AbstractType<?>> types = new ArrayList<>();
++
++        for (int i = 0; i < numberOfClusteringColumns; i++)
++            types.add(Int32Type.instance);
++
++        CompoundSparseCellNameType cType = new CompoundSparseCellNameType(types);
++        CFMetaData cfMetaData = new CFMetaData("keyspace", "test", ColumnFamilyType.Standard, cType);
++
++        for (int i = 0; i < numberOfClusteringColumns; i++)
++        {
++            ByteBuffer name = ByteBufferUtil.bytes("clustering_" + i);
++            ColumnDefinition columnDef = ColumnDefinition.clusteringKeyDef(cfMetaData, name, Int32Type.instance, i);
++            cfMetaData.addColumnDefinition(columnDef);
++        }
++        cfMetaData.rebuild();
++        return cfMetaData;
++    }
++
++    /**
++     * Creates a new <code>SingleColumnRestriction.EQ</code> instance for the specified clustering column.
++     *
++     * @param cfMetaData the column family meta data
++     * @param index the clustering column index
++     * @param value the equality value
++     * @return a new <code>SingleColumnRestriction.EQ</code> instance for the specified clustering column
++     */
++    private static Restriction newSingleEq(CFMetaData cfMetaData, int index, ByteBuffer value)
++    {
++        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
++        return new SingleColumnRestriction.EQ(columnDef, toTerm(value));
++    }
++
++    /**
++     * Creates a new <code>MultiColumnRestriction.EQ</code> instance for the specified clustering column.
++     *
++     * @param cfMetaData the column family meta data
++     * @param index the clustering column index
++     * @param value the equality value
++     * @return a new <code>MultiColumnRestriction.EQ</code> instance for the specified clustering column
++     */
++    private static Restriction newMultiEq(CFMetaData cfMetaData, int firstIndex, ByteBuffer... values)
++    {
++        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
++        for (int i = 0; i < values.length; i++)
++        {
++            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, firstIndex + i));
++        }
++        return new MultiColumnRestriction.EQ(columnDefinitions, toMultiItemTerminal(values));
++    }
++
++    /**
++     * Creates a new <code>MultiColumnRestriction.IN</code> instance for the specified clustering column.
++     *
++     * @param cfMetaData the column family meta data
++     * @param firstIndex the index of the first clustering column
++     * @param values the in values
++     * @return a new <code>MultiColumnRestriction.IN</code> instance for the specified clustering column
++     */
++    @SafeVarargs
++    private static Restriction newMultiIN(CFMetaData cfMetaData, int firstIndex, List<ByteBuffer>... values)
++    {
++        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
++        List<Term> terms = new ArrayList<>();
++        for (int i = 0; i < values.length; i++)
++        {
++            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, firstIndex + i));
++            terms.add(toMultiItemTerminal(values[i].toArray(new ByteBuffer[0])));
++        }
++        return new MultiColumnRestriction.InWithValues(columnDefinitions, terms);
++    }
++
++    /**
++     * Creates a new <code>SingleColumnRestriction.IN</code> instance for the specified clustering column.
++     *
++     * @param cfMetaData the column family meta data
++     * @param index the clustering column index
++     * @param values the in values
++     * @return a new <code>SingleColumnRestriction.IN</code> instance for the specified clustering column
++     */
++    private static Restriction newSingleIN(CFMetaData cfMetaData, int index, ByteBuffer... values)
++    {
++        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
++        return new SingleColumnRestriction.InWithValues(columnDef, toTerms(values));
++    }
++
++    /**
++     * Returns the clustering <code>ColumnDefinition</code> for the specified position.
++     *
++     * @param cfMetaData the column family meta data
++     * @param index the clustering column index
++     * @return the clustering <code>ColumnDefinition</code> for the specified position.
++     */
++    private static ColumnDefinition getClusteringColumnDefinition(CFMetaData cfMetaData, int index)
++    {
++        return cfMetaData.clusteringColumns().get(index);
++    }
++
++    /**
++     * Creates a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column.
++     *
++     * @param cfMetaData the column family meta data
++     * @param index the clustering column index
++     * @param bound the slice bound
++     * @param inclusive <code>true</code> if the bound is inclusive
++     * @param value the bound value
++     * @return a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column
++     */
++    private static Restriction newSingleSlice(CFMetaData cfMetaData, int index, Bound bound, boolean inclusive, ByteBuffer value)
++    {
++        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
++        return new SingleColumnRestriction.Slice(columnDef, bound, inclusive, toTerm(value));
++    }
++
++    /**
++     * Creates a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column.
++     *
++     * @param cfMetaData the column family meta data
++     * @param index the clustering column index
++     * @param bound the slice bound
++     * @param inclusive <code>true</code> if the bound is inclusive
++     * @param value the bound value
++     * @return a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column
++     */
++    private static Restriction newMultiSlice(CFMetaData cfMetaData, int firstIndex, Bound bound, boolean inclusive, ByteBuffer... values)
++    {
++        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
++        for (int i = 0; i < values.length; i++)
++        {
++            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, i + firstIndex));
++        }
++        return new MultiColumnRestriction.Slice(columnDefinitions, bound, inclusive, toMultiItemTerminal(values));
++    }
++
++    /**
++     * 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 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;
++    }
++}


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

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


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

Branch: refs/heads/trunk
Commit: 71778eec24e42512365cb562aecf07b5fbd28265
Parents: 258e59f 9c7a601
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Thu Mar 5 12:34:56 2015 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Thu Mar 5 12:34:56 2015 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |    2 +
 .../cassandra/cql3/MultiColumnRelation.java     |   17 +-
 .../AbstractPrimaryKeyRestrictions.java         |   12 +
 .../cql3/restrictions/AbstractRestriction.java  |    5 +-
 .../ForwardingPrimaryKeyRestrictions.java       |   25 +
 .../restrictions/MultiColumnRestriction.java    |  269 ++---
 .../restrictions/PrimaryKeyRestrictionSet.java  |  325 +++++
 .../restrictions/PrimaryKeyRestrictions.java    |    6 +-
 .../cql3/restrictions/Restriction.java          |   49 +-
 .../cql3/restrictions/RestrictionSet.java       |  252 ++++
 .../cql3/restrictions/Restrictions.java         |    4 +-
 .../SingleColumnPrimaryKeyRestrictions.java     |  327 -----
 .../restrictions/SingleColumnRestriction.java   |  111 +-
 .../restrictions/SingleColumnRestrictions.java  |  210 ----
 .../restrictions/StatementRestrictions.java     |   16 +-
 .../cql3/restrictions/TokenRestriction.java     |   21 +-
 .../cql3/statements/ModificationStatement.java  |   66 +-
 .../db/composites/CompositesBuilder.java        |  117 +-
 .../cassandra/cql3/MultiColumnRelationTest.java |  206 +++-
 .../PrimaryKeyRestrictionSetTest.java           | 1120 ++++++++++++++++++
 20 files changed, 2293 insertions(+), 867 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index 7e7a901,37eb69e..b54bdd0
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@@ -112,109 -91,54 +112,106 @@@ public class MultiColumnRelation extend
  
      /**
       * For non-IN relations, returns the Tuples.Literal or Tuples.Raw marker for a single tuple.
 +     * @return a Tuples.Literal for non-IN relations or Tuples.Raw marker for a single tuple.
       */
 -    public Term.MultiColumnRaw getValue()
 +    private Term.MultiColumnRaw getValue()
      {
 -        assert relationType != Operator.IN;
 -        return valuesOrMarker;
 +        return relationType == Operator.IN ? inMarker : valuesOrMarker;
      }
  
 -    /**
 -     * For IN relations, returns the list of Tuples.Literal instances or Tuples.Raw markers.
 -     * If a single IN marker was used, this will return null;
 -     */
 -    public List<? extends Term.MultiColumnRaw> getInValues()
 +    @Override
 +    public boolean isMultiColumn()
      {
 +        return true;
 +    }
  
 -        return inValues;
 +    @Override
 +    protected Restriction newEQRestriction(CFMetaData cfm,
 +                                           VariableSpecifications boundNames) throws InvalidRequestException
 +    {
 +        List<ColumnDefinition> receivers = receivers(cfm);
 +        Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames);
-         return new MultiColumnRestriction.EQ(cfm.comparator, receivers, term);
++        return new MultiColumnRestriction.EQ(receivers, term);
      }
  
 -    /**
 -     * For IN relations, returns the single marker for the IN values if there is one, otherwise null.
 -     */
 -    public Tuples.INRaw getInMarker()
 +    @Override
 +    protected Restriction newINRestriction(CFMetaData cfm,
 +                                           VariableSpecifications boundNames) throws InvalidRequestException
      {
 -        return inMarker;
 +        List<ColumnDefinition> receivers = receivers(cfm);
 +        List<Term> terms = toTerms(receivers, inValues, cfm.ksName, boundNames);
 +        if (terms == null)
 +        {
 +            Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames);
-             return new MultiColumnRestriction.InWithMarker(cfm.comparator, receivers, (AbstractMarker) term);
++            return new MultiColumnRestriction.InWithMarker(receivers, (AbstractMarker) term);
 +        }
-         return new MultiColumnRestriction.InWithValues(cfm.comparator, receivers, terms);
++        return new MultiColumnRestriction.InWithValues(receivers, terms);
      }
  
 -    public boolean isMultiColumn()
 +    @Override
 +    protected Restriction newSliceRestriction(CFMetaData cfm,
 +                                              VariableSpecifications boundNames,
 +                                              Bound bound,
 +                                              boolean inclusive) throws InvalidRequestException
      {
 -        return true;
 +        List<ColumnDefinition> receivers = receivers(cfm);
 +        Term term = toTerm(receivers(cfm), getValue(), cfm.ksName, boundNames);
-         return new MultiColumnRestriction.Slice(cfm.comparator, receivers, bound, inclusive, term);
++        return new MultiColumnRestriction.Slice(receivers, bound, inclusive, term);
      }
  
      @Override
 -    public String toString()
 +    protected Restriction newContainsRestriction(CFMetaData cfm,
 +                                                 VariableSpecifications boundNames,
 +                                                 boolean isKey) throws InvalidRequestException
      {
 -        if (relationType == Operator.IN)
 +        throw invalidRequest("%s cannot be used for Multi-column relations", operator());
 +    }
 +
 +    @Override
 +    protected Term toTerm(List<? extends ColumnSpecification> receivers,
 +                          Raw raw,
 +                          String keyspace,
 +                          VariableSpecifications boundNames) throws InvalidRequestException
 +    {
 +        Term term = ((MultiColumnRaw) raw).prepare(keyspace, receivers);
 +        term.collectMarkerSpecification(boundNames);
 +        return term;
 +    }
 +
 +    protected List<ColumnDefinition> receivers(CFMetaData cfm) throws InvalidRequestException
 +    {
 +        List<ColumnDefinition> names = new ArrayList<>(getEntities().size());
 +        int previousPosition = -1;
 +        for (ColumnIdentifier.Raw raw : getEntities())
          {
 -            StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
 -            sb.append(" IN ");
 -            sb.append(inMarker != null ? '?' : Tuples.tupleToString(inValues));
 -            return sb.toString();
 +            ColumnDefinition def = toColumnDefinition(cfm, raw);
 +            checkTrue(def.isClusteringColumn(), "Multi-column relations can only be applied to clustering columns but was applied to: %s", def.name);
 +            checkFalse(names.contains(def), "Column \"%s\" appeared twice in a relation: %s", def.name, this);
 +
 +            // check that no clustering columns were skipped
-             if (def.position() != previousPosition + 1)
-             {
-                 checkFalse(previousPosition == -1, "Clustering columns may not be skipped in multi-column relations. " +
-                                                    "They should appear in the PRIMARY KEY order. Got %s", this);
-                 throw invalidRequest("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", this);
-             }
++            checkFalse(previousPosition != -1 && def.position() != previousPosition + 1,
++                       "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", this);
++
 +            names.add(def);
 +            previousPosition = def.position();
          }
 -        else
 +        return names;
 +    }
 +
 +    @Override
 +    public String toString()
 +    {
 +        StringBuilder builder = new StringBuilder(Tuples.tupleToString(entities));
 +        if (isIN())
          {
 -            StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
 -            sb.append(" ");
 -            sb.append(relationType);
 -            sb.append(" ");
 -            sb.append(valuesOrMarker);
 -            return sb.toString();
 +            return builder.append(" IN ")
 +                          .append(inMarker != null ? '?' : Tuples.tupleToString(inValues))
 +                          .toString();
          }
 +        return builder.append(" ")
 +                      .append(relationType)
 +                      .append(" ")
 +                      .append(valuesOrMarker)
 +                      .toString();
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
index 0107603,0000000..2eaa386
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
@@@ -1,48 -1,0 +1,60 @@@
 +/*
 + * 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.restrictions;
 +
++import java.nio.ByteBuffer;
++import java.util.List;
++
++import org.apache.cassandra.cql3.QueryOptions;
++import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.composites.CType;
++import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * Base class for <code>PrimaryKeyRestrictions</code>.
 + */
 +abstract class AbstractPrimaryKeyRestrictions extends AbstractRestriction implements PrimaryKeyRestrictions
 +{
 +    /**
 +     * The composite type.
 +     */
 +    protected final CType ctype;
 +
 +    public AbstractPrimaryKeyRestrictions(CType ctype)
 +    {
 +        this.ctype = ctype;
 +    }
 +
 +    @Override
++    public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
++    {
++        return values(options);
++    }
++
++    @Override
 +    public final boolean isEmpty()
 +    {
 +        return getColumnDefs().isEmpty();
 +    }
 +
 +    @Override
 +    public final int size()
 +    {
 +        return getColumnDefs().size();
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
index 0ae7b22,0000000..b212f4d
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
@@@ -1,129 -1,0 +1,130 @@@
 +/*
 + * 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.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.List;
 +
 +import org.apache.cassandra.cql3.ColumnSpecification;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.cql3.statements.Bound;
++import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
 +
 +/**
 + * Base class for <code>Restriction</code>s
 + */
 +abstract class AbstractRestriction  implements Restriction
 +{
 +    @Override
 +    public  boolean isOnToken()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isMultiColumn()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isSlice()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isEQ()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isIN()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isContains()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean hasBound(Bound b)
 +    {
 +        return true;
 +    }
 +
 +    @Override
-     public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
++    public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +    {
-         return values(options);
++        return appendTo(builder, options);
 +    }
 +
 +    @Override
 +    public boolean isInclusive(Bound b)
 +    {
 +        return true;
 +    }
 +
 +    protected static ByteBuffer validateIndexedValue(ColumnSpecification columnSpec,
 +                                                     ByteBuffer value)
 +                                                     throws InvalidRequestException
 +    {
 +        checkNotNull(value, "Unsupported null value for indexed column %s", columnSpec.name);
 +        checkFalse(value.remaining() > 0xFFFF, "Index expression values may not be larger than 64K");
 +        return value;
 +    }
 +
 +    /**
 +     * Checks if the specified term is using the specified function.
 +     *
 +     * @param term the term to check
 +     * @param ksName the function keyspace name
 +     * @param functionName the function name
 +     * @return <code>true</code> if the specified term is using the specified function, <code>false</code> otherwise.
 +     */
 +    protected static final boolean usesFunction(Term term, String ksName, String functionName)
 +    {
 +        return term != null && term.usesFunction(ksName, functionName);
 +    }
 +
 +    /**
 +     * Checks if one of the specified term is using the specified function.
 +     *
 +     * @param terms the terms to check
 +     * @param ksName the function keyspace name
 +     * @param functionName the function name
 +     * @return <code>true</code> if onee of the specified term is using the specified function, <code>false</code> otherwise.
 +     */
 +    protected static final boolean usesFunction(List<Term> terms, String ksName, String functionName)
 +    {
 +        if (terms != null)
 +            for (Term value : terms)
 +                if (usesFunction(value, ksName, functionName))
 +                    return true;
 +        return false;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
index 5492c2b,0000000..537481f
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
@@@ -1,160 -1,0 +1,185 @@@
 +/*
 + * 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.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Collection;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
 +import org.apache.cassandra.db.composites.Composite;
++import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * A <code>PrimaryKeyRestrictions</code> which forwards all its method calls to another 
 + * <code>PrimaryKeyRestrictions</code>. Subclasses should override one or more methods to modify the behavior 
 + * of the backing <code>PrimaryKeyRestrictions</code> as desired per the decorator pattern. 
 + */
 +abstract class ForwardingPrimaryKeyRestrictions implements PrimaryKeyRestrictions
 +{
 +    /**
 +     * Returns the backing delegate instance that methods are forwarded to.
 +     * @return the backing delegate instance that methods are forwarded to.
 +     */
 +    protected abstract PrimaryKeyRestrictions getDelegate();
 +
 +    @Override
 +    public boolean usesFunction(String ksName, String functionName)
 +    {
 +        return getDelegate().usesFunction(ksName, functionName);
 +    }
 +
 +    @Override
 +    public Collection<ColumnDefinition> getColumnDefs()
 +    {
 +        return getDelegate().getColumnDefs();
 +    }
 +
 +    @Override
++    public ColumnDefinition getFirstColumn()
++    {
++        return getDelegate().getFirstColumn();
++    }
++
++    @Override
++    public ColumnDefinition getLastColumn()
++    {
++        return getDelegate().getLastColumn();
++    }
++
++    @Override
 +    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
 +    {
 +        return getDelegate().mergeWith(restriction);
 +    }
 +
 +    @Override
 +    public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager)
 +    {
 +        return getDelegate().hasSupportingIndex(secondaryIndexManager);
 +    }
 +
 +    @Override
 +    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
 +    {
 +        return getDelegate().values(options);
 +    }
 +
 +    @Override
++    public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
++    {
++        return getDelegate().appendTo(builder, options);
++    }
++
++    @Override
 +    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
 +    {
 +        return getDelegate().valuesAsComposites(options);
 +    }
 +
 +    @Override
 +    public List<ByteBuffer> bounds(Bound bound, QueryOptions options) throws InvalidRequestException
 +    {
 +        return getDelegate().bounds(bound, options);
 +    }
 +
 +    @Override
 +    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
 +    {
 +        return getDelegate().boundsAsComposites(bound, options);
 +    }
 +
 +    @Override
++    public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
++    {
++        return getDelegate().appendBoundTo(builder, bound, options);
++    }
++
++    @Override
 +    public boolean isInclusive(Bound bound)
 +    {
 +        return getDelegate().isInclusive(bound.reverse());
 +    }
 +
 +    @Override
 +    public boolean isEmpty()
 +    {
 +        return getDelegate().isEmpty();
 +    }
 +
 +    @Override
 +    public int size()
 +    {
 +        return getDelegate().size();
 +    }
 +
 +    @Override
 +    public boolean isOnToken()
 +    {
 +        return getDelegate().isOnToken();
 +    }
 +
 +    @Override
 +    public boolean isSlice()
 +    {
 +        return getDelegate().isSlice();
 +    }
 +
 +    @Override
 +    public boolean isEQ()
 +    {
 +        return getDelegate().isEQ();
 +    }
 +
 +    @Override
 +    public boolean isIN()
 +    {
 +        return getDelegate().isIN();
 +    }
 +
 +    @Override
 +    public boolean isContains()
 +    {
 +        return getDelegate().isContains();
 +    }
 +
 +    @Override
 +    public boolean isMultiColumn()
 +    {
 +        return getDelegate().isMultiColumn();
 +    }
 +
 +    @Override
 +    public boolean hasBound(Bound b)
 +    {
 +        return getDelegate().hasBound(b);
 +    }
 +
 +    @Override
 +    public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                     SecondaryIndexManager indexManager,
 +                                     QueryOptions options) throws InvalidRequestException
 +    {
 +        getDelegate().addIndexExpressionTo(expressions, indexManager, options);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
index 9f6ab4c,0000000..be6d905
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
@@@ -1,519 -1,0 +1,452 @@@
 +/*
 + * 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.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.AbstractMarker;
 +import org.apache.cassandra.cql3.Operator;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.cql3.Tuples;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
- import org.apache.cassandra.db.composites.CBuilder;
- import org.apache.cassandra.db.composites.CType;
- import org.apache.cassandra.db.composites.Composite;
- import org.apache.cassandra.db.composites.Composites;
++import org.apache.cassandra.db.composites.*;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
- public abstract class MultiColumnRestriction extends AbstractPrimaryKeyRestrictions
++public abstract class MultiColumnRestriction extends AbstractRestriction
 +{
 +    /**
 +     * The columns to which the restriction apply.
 +     */
 +    protected final List<ColumnDefinition> columnDefs;
 +
-     public MultiColumnRestriction(CType ctype, List<ColumnDefinition> columnDefs)
++    public MultiColumnRestriction(List<ColumnDefinition> columnDefs)
 +    {
-         super(ctype);
 +        this.columnDefs = columnDefs;
 +    }
 +
 +    @Override
 +    public boolean isMultiColumn()
 +    {
 +        return true;
 +    }
 +
 +    @Override
-     public Collection<ColumnDefinition> getColumnDefs()
++    public ColumnDefinition getFirstColumn()
 +    {
-         return columnDefs;
++        return columnDefs.get(0);
++    }
++
++    @Override
++    public ColumnDefinition getLastColumn()
++    {
++        return columnDefs.get(columnDefs.size() - 1);
 +    }
 +
 +    @Override
-     public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++    public Collection<ColumnDefinition> getColumnDefs()
 +    {
-         return Composites.toByteBuffers(valuesAsComposites(options));
++        return columnDefs;
 +    }
 +
 +    @Override
-     public final PrimaryKeyRestrictions mergeWith(Restriction otherRestriction) throws InvalidRequestException
++    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
 +    {
 +            checkTrue(otherRestriction.isMultiColumn(),
 +                      "Mixing single column relations and multi column relations on clustering columns is not allowed");
-             return doMergeWith((PrimaryKeyRestrictions) otherRestriction);
++            return doMergeWith(otherRestriction);
 +    }
 +
-     protected abstract PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException;
++    protected abstract Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException;
 +
 +    /**
 +     * Returns the names of the columns that are specified within this <code>Restrictions</code> and the other one
 +     * as a comma separated <code>String</code>.
 +     *
 +     * @param otherRestrictions the other restrictions
 +     * @return the names of the columns that are specified within this <code>Restrictions</code> and the other one
 +     * as a comma separated <code>String</code>.
 +     */
-     protected final String getColumnsInCommons(Restrictions otherRestrictions)
++    protected final String getColumnsInCommons(Restriction otherRestriction)
 +    {
 +        Set<ColumnDefinition> commons = new HashSet<>(getColumnDefs());
-         commons.retainAll(otherRestrictions.getColumnDefs());
++        commons.retainAll(otherRestriction.getColumnDefs());
 +        StringBuilder builder = new StringBuilder();
 +        for (ColumnDefinition columnDefinition : commons)
 +        {
 +            if (builder.length() != 0)
 +                builder.append(" ,");
 +            builder.append(columnDefinition.name);
 +        }
 +        return builder.toString();
 +    }
 +
 +    @Override
 +    public final boolean hasSupportingIndex(SecondaryIndexManager indexManager)
 +    {
 +        for (ColumnDefinition columnDef : columnDefs)
 +        {
 +            SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
 +            if (index != null && isSupportedBy(index))
 +                return true;
 +        }
 +        return false;
 +    }
 +
-     @Override
-     public final void addIndexExpressionTo(List<IndexExpression> expressions,
-                                            SecondaryIndexManager indexManager,
-                                            QueryOptions options) throws InvalidRequestException
-     {
-         for (ColumnDefinition columnDef : columnDefs)
-         {
-             SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
-             if (index != null && isSupportedBy(index))
-                 expressions.add(getIndexExpression(columnDef, options));
-         }
-     }
- 
-     /**
-      * Returns the <code>IndexExpression</code> for the specified column.
-      *
-      * @param columnDef the column definition
-      * @param options the query options
-      * @return the <code>IndexExpression</code> for the specified column
-      */
-     protected IndexExpression getIndexExpression(ColumnDefinition columnDef,
-                                                  QueryOptions options) throws InvalidRequestException
-     {
-         // Except for EQ this operation is not supported
-         throw new UnsupportedOperationException();
-     }
- 
 +    /**
 +     * Check if this type of restriction is supported for by the specified index.
 +     * @param index the Secondary index
 +     *
 +     * @return <code>true</code> this type of restriction is supported by the specified index,
 +     * <code>false</code> otherwise.
 +     */
 +    protected abstract boolean isSupportedBy(SecondaryIndex index);
 +
 +    public static class EQ  extends MultiColumnRestriction
 +    {
 +        protected final Term value;
 +
-         public EQ(CType ctype, List<ColumnDefinition> columnDefs, Term value)
++        public EQ(List<ColumnDefinition> columnDefs, Term value)
 +        {
-             super(ctype, columnDefs);
++            super(columnDefs);
 +            this.value = value;
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return usesFunction(value, ksName, functionName);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("EQ(%s)", value);
 +        }
 +
 +        @Override
-         public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException
++        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
 +                                 getColumnsInCommons(otherRestriction));
 +        }
 +
 +        @Override
-         public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
-         {
-             return Collections.singletonList(compositeValue(options));
-         }
- 
-         @Override
-         public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
-         {
-             Composite prefix = compositeValue(options);
-             return Collections.singletonList(ctype.size() > prefix.size() && bound.isEnd()
-                                              ? prefix.end()
-                                              : prefix);
-         }
- 
-         @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.EQ);
 +        }
 +
-         private Composite compositeValue(QueryOptions options) throws InvalidRequestException
++        @Override
++        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
-             CBuilder builder = ctype.builder();
 +            Tuples.Value t = ((Tuples.Value) value.bind(options));
 +            List<ByteBuffer> values = t.getElements();
-             for (int i = 0; i < values.size(); i++)
++            for (int i = 0, m = values.size(); i < m; i++)
 +            {
-                 ByteBuffer component = checkNotNull(values.get(i),
-                                                     "Invalid null value in condition for column %s",
-                                                     columnDefs.get(i).name);
-                 builder.add(component);
++                builder.addElementToAll(values.get(i));
++                checkFalse(builder.containsNull(), "Invalid null value for column %s", columnDefs.get(i).name);
 +            }
- 
-             return builder.build();
++            return builder;
 +        }
 +
 +        @Override
-         protected final IndexExpression getIndexExpression(ColumnDefinition columnDef,
-                                                            QueryOptions options) throws InvalidRequestException
++        public final void addIndexExpressionTo(List<IndexExpression> expressions,
++                                               SecondaryIndexManager indexManager,
++                                               QueryOptions options) throws InvalidRequestException
 +        {
 +            Tuples.Value t = ((Tuples.Value) value.bind(options));
 +            List<ByteBuffer> values = t.getElements();
-             ByteBuffer component = validateIndexedValue(columnDef, values.get(columnDef.position()));
-             return new IndexExpression(columnDef.name.bytes, Operator.EQ, component);
-         }
-     }
 +
-     public abstract static class IN extends MultiColumnRestriction
-     {
-         @Override
-         public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
-         {
-             CBuilder builder = ctype.builder();
-             List<List<ByteBuffer>> splitInValues = splitValues(options);
-             // The IN query might not have listed the values in comparator order, so we need to re-sort
-             // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
-             TreeSet<Composite> inValues = new TreeSet<>(ctype);
-             for (List<ByteBuffer> components : splitInValues)
++            for (int i = 0, m = columnDefs.size(); i < m; i++)
 +            {
-                 for (int i = 0; i < components.size(); i++)
-                     checkNotNull(components.get(i), "Invalid null value in condition for column " + columnDefs.get(i).name);
- 
-                 inValues.add(builder.buildWith(components));
++                ColumnDefinition columnDef = columnDefs.get(i);
++                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
++                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
 +            }
-             return new ArrayList<>(inValues);
 +        }
++    }
 +
++    public abstract static class IN extends MultiColumnRestriction
++    {
++        /**
++         * {@inheritDoc}
++         */
 +        @Override
-         public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
++        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
-             CBuilder builder = ctype.builder();
 +            List<List<ByteBuffer>> splitInValues = splitValues(options);
-             // The IN query might not have listed the values in comparator order, so we need to re-sort
-             // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
-             TreeSet<Composite> inValues = new TreeSet<>(ctype);
-             for (List<ByteBuffer> components : splitInValues)
-             {
-                 for (int i = 0; i < components.size(); i++)
-                     checkNotNull(components.get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
++            builder.addAllElementsToAll(splitInValues);
 +
-                 Composite prefix = builder.buildWith(components);
-                 inValues.add(bound.isEnd() && builder.remainingCount() - components.size() > 0
-                              ? prefix.end()
-                              : prefix);
-             }
-             return new ArrayList<>(inValues);
++            if (builder.containsNull())
++                throw invalidRequest("Invalid null value in condition for columns: %s", ColumnDefinition.toIdentifiers(columnDefs));
++            return builder;
 +        }
 +
-         public IN(CType ctype, List<ColumnDefinition> columnDefs)
++        public IN(List<ColumnDefinition> columnDefs)
 +        {
-             super(ctype, columnDefs);
++            super(columnDefs);
 +        }
 +
 +        @Override
 +        public boolean isIN()
 +        {
 +            return true;
 +        }
 +
 +        @Override
-         public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestrictions) throws InvalidRequestException
++        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN",
-                                  getColumnsInCommons(otherRestrictions));
++                                 getColumnsInCommons(otherRestriction));
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.IN);
 +        }
 +
++        @Override
++        public final void addIndexExpressionTo(List<IndexExpression> expressions,
++                                               SecondaryIndexManager indexManager,
++                                               QueryOptions options) throws InvalidRequestException
++        {
++            List<List<ByteBuffer>> splitInValues = splitValues(options);
++            checkTrue(splitInValues.size() == 1, "IN restrictions are not supported on indexed columns");
++            List<ByteBuffer> values = splitInValues.get(0);
++
++            for (int i = 0, m = columnDefs.size(); i < m; i++)
++            {
++                ColumnDefinition columnDef = columnDefs.get(i);
++                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
++                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
++            }
++        }
++
 +        protected abstract List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
 +    }
 +
 +    /**
 +     * An IN restriction that has a set of terms for in values.
 +     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
 +     */
 +    public static class InWithValues extends MultiColumnRestriction.IN
 +    {
 +        protected final List<Term> values;
 +
-         public InWithValues(CType ctype, List<ColumnDefinition> columnDefs, List<Term> values)
++        public InWithValues(List<ColumnDefinition> columnDefs, List<Term> values)
 +        {
-             super(ctype, columnDefs);
++            super(columnDefs);
 +            this.values = values;
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return usesFunction(values, ksName, functionName);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("IN(%s)", values);
 +        }
 +
 +        @Override
 +        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
 +            for (Term value : values)
 +            {
 +                Term.MultiItemTerminal term = (Term.MultiItemTerminal) value.bind(options);
 +                buffers.add(term.getElements());
 +            }
 +            return buffers;
 +        }
 +    }
 +
 +    /**
 +     * An IN restriction that uses a single marker for a set of IN values that are tuples.
 +     * For example: "SELECT ... WHERE (a, b, c) IN ?"
 +     */
 +    public static class InWithMarker extends MultiColumnRestriction.IN
 +    {
 +        protected final AbstractMarker marker;
 +
-         public InWithMarker(CType ctype, List<ColumnDefinition> columnDefs, AbstractMarker marker)
++        public InWithMarker(List<ColumnDefinition> columnDefs, AbstractMarker marker)
 +        {
-             super(ctype, columnDefs);
++            super(columnDefs);
 +            this.marker = marker;
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return false;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "IN ?";
 +        }
 +
 +        @Override
 +        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            Tuples.InMarker inMarker = (Tuples.InMarker) marker;
 +            Tuples.InValue inValue = inMarker.bind(options);
 +            checkNotNull(inValue, "Invalid null value for IN restriction");
 +            return inValue.getSplitValues();
 +        }
 +    }
 +
 +    public static class Slice extends MultiColumnRestriction
 +    {
 +        private final TermSlice slice;
 +
-         public Slice(CType ctype, List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
++        public Slice(List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
 +        {
-             this(ctype, columnDefs, TermSlice.newInstance(bound, inclusive, term));
++            this(columnDefs, TermSlice.newInstance(bound, inclusive, term));
 +        }
 +
-         private Slice(CType ctype, List<ColumnDefinition> columnDefs, TermSlice slice)
++        private Slice(List<ColumnDefinition> columnDefs, TermSlice slice)
 +        {
-             super(ctype, columnDefs);
++            super(columnDefs);
 +            this.slice = slice;
 +        }
 +
 +        @Override
 +        public boolean isSlice()
 +        {
 +            return true;
 +        }
 +
 +        @Override
-         public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
++        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
-         public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
++        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
-             return Composites.toByteBuffers(boundsAsComposites(b, options));
-         }
++            List<ByteBuffer> vals = componentBounds(bound, options);
 +
-         @Override
-         public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
-         {
-             CBuilder builder = ctype.builder();
-             Iterator<ColumnDefinition> iter = columnDefs.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 = !firstName.isReversedType() ? bound : bound.reverse();
- 
-             if (!hasBound(firstComponentBound))
-             {
-                 Composite prefix = builder.build();
-                 return Collections.singletonList(builder.remainingCount() > 0 && bound.isEnd()
-                         ? prefix.end()
-                         : prefix);
-             }
- 
-             List<ByteBuffer> vals = componentBounds(firstComponentBound, options);
- 
-             ByteBuffer v = checkNotNull(vals.get(firstName.position()), "Invalid null value in condition for column %s", firstName.name);
-             builder.add(v);
- 
-             while (iter.hasNext())
++            for (int i = 0, m = vals.size(); i < m; i++)
 +            {
-                 ColumnDefinition def = iter.next();
-                 if (def.position() >= vals.size())
-                     break;
- 
-                 v = checkNotNull(vals.get(def.position()), "Invalid null value in condition for column %s", def.name);
-                 builder.add(v);
++                ByteBuffer v = checkNotNull(vals.get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
++                builder.addElementToAll(v);
 +            }
-             Composite.EOC eoc =  eocFor(this, bound, firstComponentBound);
-             return Collections.singletonList(builder.build().withEOC(eoc));
++            return builder;
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return slice.isSupportedBy(index);
 +        }
 +
-         private static Composite.EOC eocFor(Restriction r, Bound eocBound, Bound inclusiveBound)
-         {
-             if (eocBound.isStart())
-                 return r.isInclusive(inclusiveBound) ? Composite.EOC.NONE : Composite.EOC.END;
- 
-             return r.isInclusive(inclusiveBound) ? Composite.EOC.END : Composite.EOC.START;
-         }
- 
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            return slice.hasBound(b);
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName))
 +                    || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName));
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            return slice.isInclusive(b);
 +        }
 +
 +        @Override
-         public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException
++        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isSlice(),
 +                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
 +                      getColumnsInCommons(otherRestriction));
 +
 +            Slice otherSlice = (Slice) otherRestriction;
 +
++            if (!getFirstColumn().equals(otherRestriction.getFirstColumn()))
++            {
++                ColumnDefinition column = getFirstColumn().position() > otherRestriction.getFirstColumn().position()
++                        ? getFirstColumn() : otherRestriction.getFirstColumn();
++
++                throw invalidRequest("Column \"%s\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
++                                     column.name);
++            }
++
 +            checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
 +                       "More than one restriction was found for the start bound on %s",
 +                       getColumnsInCommons(otherRestriction));
 +            checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
 +                       "More than one restriction was found for the end bound on %s",
 +                       getColumnsInCommons(otherRestriction));
 +
-             List<ColumnDefinition> newColumnDefs = size() >= otherSlice.size() ?  columnDefs : otherSlice.columnDefs;
-             return new Slice(ctype,  newColumnDefs, slice.merge(otherSlice.slice));
++            List<ColumnDefinition> newColumnDefs = columnDefs.size() >= otherSlice.columnDefs.size() ?  columnDefs : otherSlice.columnDefs;
++            return new Slice(newColumnDefs, slice.merge(otherSlice.slice));
++        }
++
++        @Override
++        public final void addIndexExpressionTo(List<IndexExpression> expressions,
++                                               SecondaryIndexManager indexManager,
++                                               QueryOptions options) throws InvalidRequestException
++        {
++            throw invalidRequest("Slice restrictions are not supported on indexed columns");
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "SLICE" + slice;
 +        }
 +
 +        /**
 +         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
 +         * ByteBuffer to represent the entire bound.
 +         * @param b the bound type
 +         * @param options the query options
 +         * @return one ByteBuffer per-component in the bound
 +         * @throws InvalidRequestException if the components cannot be retrieved
 +         */
 +        private List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
 +        {
 +            Tuples.Value value = (Tuples.Value) slice.bound(b).bind(options);
 +            return value.getElements();
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
index 0000000,0000000..194f4d5
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
@@@ -1,0 -1,0 +1,325 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++package org.apache.cassandra.cql3.restrictions;
++
++import java.nio.ByteBuffer;
++import java.util.Collection;
++import java.util.Collections;
++import java.util.List;
++
++import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.QueryOptions;
++import org.apache.cassandra.cql3.statements.Bound;
++import org.apache.cassandra.db.IndexExpression;
++import org.apache.cassandra.db.composites.*;
++import org.apache.cassandra.db.composites.Composite.EOC;
++import org.apache.cassandra.db.index.SecondaryIndexManager;
++import org.apache.cassandra.exceptions.InvalidRequestException;
++
++import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
++import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
++
++/**
++ * A set of single column restrictions on a primary key part (partition key or clustering key).
++ */
++final class PrimaryKeyRestrictionSet extends AbstractPrimaryKeyRestrictions
++{
++    /**
++     * The restrictions.
++     */
++    private final RestrictionSet restrictions;
++
++    /**
++     * <code>true</code> if the restrictions are corresponding to an EQ, <code>false</code> otherwise.
++     */
++    private boolean eq;
++
++    /**
++     * <code>true</code> if the restrictions are corresponding to an IN, <code>false</code> otherwise.
++     */
++    private boolean in;
++
++    /**
++     * <code>true</code> if the restrictions are corresponding to a Slice, <code>false</code> otherwise.
++     */
++    private boolean slice;
++
++    /**
++     * <code>true</code> if the restrictions are corresponding to a Contains, <code>false</code> otherwise.
++     */
++    private boolean contains;
++
++    public PrimaryKeyRestrictionSet(CType ctype)
++    {
++        super(ctype);
++        this.restrictions = new RestrictionSet();
++        this.eq = true;
++    }
++
++    private PrimaryKeyRestrictionSet(PrimaryKeyRestrictionSet primaryKeyRestrictions,
++                                               Restriction restriction) throws InvalidRequestException
++    {
++        super(primaryKeyRestrictions.ctype);
++        this.restrictions = primaryKeyRestrictions.restrictions.addRestriction(restriction);
++
++        if (!primaryKeyRestrictions.isEmpty())
++        {
++            ColumnDefinition lastRestrictionStart = primaryKeyRestrictions.restrictions.lastRestriction().getFirstColumn();
++            ColumnDefinition newRestrictionStart = restriction.getFirstColumn();
++
++            checkFalse(primaryKeyRestrictions.isSlice() && newRestrictionStart.position() > lastRestrictionStart.position(),
++                       "Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
++                       newRestrictionStart.name,
++                       lastRestrictionStart.name);
++
++            if (newRestrictionStart.position() < lastRestrictionStart.position() && restriction.isSlice())
++                throw invalidRequest("PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
++                                     restrictions.nextColumn(newRestrictionStart).name,
++                                     newRestrictionStart.name);
++        }
++
++        if (restriction.isSlice() || primaryKeyRestrictions.isSlice())
++            this.slice = true;
++        else if (restriction.isContains() || primaryKeyRestrictions.isContains())
++            this.contains = true;
++        else if (restriction.isIN())
++            this.in = true;
++        else
++            this.eq = true;
++    }
++
++    @Override
++    public boolean isSlice()
++    {
++        return slice;
++    }
++
++    @Override
++    public boolean isEQ()
++    {
++        return eq;
++    }
++
++    @Override
++    public boolean isIN()
++    {
++        return in;
++    }
++
++    @Override
++    public boolean isOnToken()
++    {
++        return false;
++    }
++
++    @Override
++    public boolean isContains()
++    {
++        return contains;
++    }
++
++    @Override
++    public boolean isMultiColumn()
++    {
++        return false;
++    }
++
++    @Override
++    public boolean usesFunction(String ksName, String functionName)
++    {
++        return restrictions.usesFunction(ksName, functionName);
++    }
++
++    @Override
++    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
++    {
++        if (restriction.isOnToken())
++        {
++            if (isEmpty())
++                return (PrimaryKeyRestrictions) restriction;
++
++            return new TokenFilter(this, (TokenRestriction) restriction);
++        }
++
++        return new PrimaryKeyRestrictionSet(this, restriction);
++    }
++
++    @Override
++    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
++    {
++        return appendTo(new CompositesBuilder(ctype), options).build();
++    }
++
++    @Override
++    public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
++    {
++        for (Restriction r : restrictions)
++        {
++            r.appendTo(builder, options);
++            if (builder.hasMissingElements())
++                break;
++        }
++        return builder;
++    }
++
++    @Override
++    public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
++    {
++        throw new UnsupportedOperationException();
++    }
++
++    @Override
++    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
++    {
++        CompositesBuilder builder = new CompositesBuilder(ctype);
++        // 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
++        int keyPosition = 0;
++        for (Restriction r : restrictions)
++        {
++            ColumnDefinition def = r.getFirstColumn();
++
++            // 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.
++            // But if the actual comparator itself is reversed, we must inversed the bounds too.
++            Bound b = !def.isReversedType() ? bound : bound.reverse();
++            if (keyPosition != def.position() || r.isContains())
++                break;
++
++            if (r.isSlice())
++            {
++                if (!r.hasBound(b))
++                {
++                    // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix.
++                    // For composites, if there was preceding component and we're computing the end, we must change the last component
++                    // End-Of-Component, otherwise we would be selecting only one record.
++                    return builder.buildWithEOC(bound.isEnd() ? EOC.END : EOC.START);
++                }
++
++                r.appendBoundTo(builder, b, options);
++                Composite.EOC eoc = eocFor(r, bound, b);
++                return builder.buildWithEOC(eoc);
++            }
++
++            r.appendBoundTo(builder, b, options);
++
++            if (builder.hasMissingElements())
++                return Collections.emptyList();
++
++            keyPosition = r.getLastColumn().position() + 1;
++        }
++        // 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
++        // prefix.end() will sort after prefix (see #5240).
++        EOC eoc = !builder.hasRemaining() ? EOC.NONE : (bound.isEnd() ? EOC.END : EOC.START);
++        return builder.buildWithEOC(eoc);
++    }
++
++    @Override
++    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++    {
++        return Composites.toByteBuffers(valuesAsComposites(options));
++    }
++
++    @Override
++    public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
++    {
++        return Composites.toByteBuffers(boundsAsComposites(b, options));
++    }
++
++    private static Composite.EOC eocFor(Restriction r, Bound eocBound, Bound inclusiveBound)
++    {
++        if (eocBound.isStart())
++            return r.isInclusive(inclusiveBound) ? Composite.EOC.NONE : Composite.EOC.END;
++
++        return r.isInclusive(inclusiveBound) ? Composite.EOC.END : Composite.EOC.START;
++    }
++
++    @Override
++    public boolean hasBound(Bound b)
++    {
++        if (isEmpty())
++            return false;
++        return restrictions.lastRestriction().hasBound(b);
++    }
++
++    @Override
++    public boolean isInclusive(Bound b)
++    {
++        if (isEmpty())
++            return false;
++        return restrictions.lastRestriction().isInclusive(b);
++    }
++
++    @Override
++    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
++    {
++        return restrictions.hasSupportingIndex(indexManager);
++    }
++
++    @Override
++    public void addIndexExpressionTo(List<IndexExpression> expressions,
++                                     SecondaryIndexManager indexManager,
++                                     QueryOptions options) throws InvalidRequestException
++    {
++        Boolean clusteringColumns = null;
++        int position = 0;
++
++        for (Restriction restriction : restrictions)
++        {
++            ColumnDefinition columnDef = restriction.getFirstColumn();
++
++            // PrimaryKeyRestrictionSet contains only one kind of column, either partition key or clustering columns.
++            // Therefore we only need to check the column kind once. All the other columns will be of the same kind.
++            if (clusteringColumns == null)
++                clusteringColumns = columnDef.isClusteringColumn() ? Boolean.TRUE : Boolean.FALSE;
++
++            // We ignore all the clustering columns that can be handled by slices.
++            if (clusteringColumns && !restriction.isContains()&& position == columnDef.position())
++            {
++                position = restriction.getLastColumn().position() + 1;
++                if (!restriction.hasSupportingIndex(indexManager))
++                    continue;
++            }
++            restriction.addIndexExpressionTo(expressions, indexManager, options);
++        }
++    }
++
++    @Override
++    public Collection<ColumnDefinition> getColumnDefs()
++    {
++        return restrictions.getColumnDefs();
++    }
++
++    @Override
++    public ColumnDefinition getFirstColumn()
++    {
++        return restrictions.firstColumn();
++    }
++
++    @Override
++    public ColumnDefinition getLastColumn()
++    {
++        return restrictions.lastColumn();
++    }
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
index 5f977b7,0000000..7d7b492
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
@@@ -1,40 -1,0 +1,44 @@@
 +/*
 + * 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.restrictions;
 +
++import java.nio.ByteBuffer;
 +import java.util.List;
 +
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * A set of restrictions on a primary key part (partition key or clustering key).
 + *
 + */
 +interface PrimaryKeyRestrictions extends Restriction, Restrictions
 +{
- 
 +    @Override
 +    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException;
 +
++    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
++
 +    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException;
 +
++    public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException;
++
 +    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException;
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
index f6d0c73,0000000..49af20c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
@@@ -1,99 -1,0 +1,140 @@@
 +/*
 + * 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.restrictions;
 +
- import java.nio.ByteBuffer;
++import java.util.Collection;
 +import java.util.List;
 +
++import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
++import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * A restriction/clause on a column.
 + * The goal of this class being to group all conditions for a column in a SELECT.
++ *
++ * <p>Implementation of this class must be immutable. See {@link #mergeWith(Restriction)} for more explanation.</p>
 + */
 +public interface Restriction
 +{
 +    public boolean isOnToken();
 +    public boolean isSlice();
 +    public boolean isEQ();
 +    public boolean isIN();
 +    public boolean isContains();
 +    public boolean isMultiColumn();
 +
-     public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
++    /**
++     * Returns the definition of the first column.
++     * @return the definition of the first column.
++     */
++    public ColumnDefinition getFirstColumn();
++
++    /**
++     * Returns the definition of the last column.
++     * @return the definition of the last column.
++     */
++    public ColumnDefinition getLastColumn();
++
++    /**
++     * Returns the column definitions in position order.
++     * @return the column definitions in position order.
++     */
++    public Collection<ColumnDefinition> getColumnDefs();
 +
 +    /**
 +     * Returns <code>true</code> if one of the restrictions use the specified function.
 +     *
 +     * @param ksName the keyspace name
 +     * @param functionName the function name
 +     * @return <code>true</code> if one of the restrictions use the specified function, <code>false</code> otherwise.
 +     */
 +    public boolean usesFunction(String ksName, String functionName);
 +
 +    /**
 +     * Checks if the specified bound is set or not.
 +     * @param b the bound type
 +     * @return <code>true</code> if the specified bound is set, <code>false</code> otherwise
 +     */
 +    public boolean hasBound(Bound b);
 +
-     public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException;
- 
 +    /**
 +     * Checks if the specified bound is inclusive or not.
 +     * @param b the bound type
 +     * @return <code>true</code> if the specified bound is inclusive, <code>false</code> otherwise
 +     */
 +    public boolean isInclusive(Bound b);
 +
 +    /**
 +     * Merges this restriction with the specified one.
 +     *
++     * <p>Restriction are immutable. Therefore merging two restrictions result in a new one.
++     * The reason behind this choice is that it allow a great flexibility in the way the merging can done while
++     * preventing any side effect.</p>
++     *
 +     * @param otherRestriction the restriction to merge into this one
 +     * @return the restriction resulting of the merge
 +     * @throws InvalidRequestException if the restrictions cannot be merged
 +     */
 +    public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException;
 +
 +    /**
 +     * Check if the restriction is on indexed columns.
 +     *
 +     * @param indexManager the index manager
 +     * @return <code>true</code> if the restriction is on indexed columns, <code>false</code>
 +     */
 +    public boolean hasSupportingIndex(SecondaryIndexManager indexManager);
 +
 +    /**
 +     * Adds to the specified list the <code>IndexExpression</code>s corresponding to this <code>Restriction</code>.
 +     *
 +     * @param expressions the list to add the <code>IndexExpression</code>s to
 +     * @param indexManager the secondary index manager
 +     * @param options the query options
 +     * @throws InvalidRequestException if this <code>Restriction</code> cannot be converted into 
 +     * <code>IndexExpression</code>s
 +     */
 +    public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                     SecondaryIndexManager indexManager,
 +                                     QueryOptions options)
 +                                     throws InvalidRequestException;
++
++    /**
++     * Appends the values of this <code>Restriction</code> to the specified builder.
++     *
++     * @param builder the <code>CompositesBuilder</code> to append to.
++     * @param options the query options
++     * @return the <code>CompositesBuilder</code>
++     */
++    public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options);
++
++    /**
++     * Appends the values of the <code>Restriction</code> for the specified bound to the specified builder.
++     *
++     * @param builder the <code>CompositesBuilder</code> to append to.
++     * @param bound the bound
++     * @param options the query options
++     * @return the <code>CompositesBuilder</code>
++     */
++    public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options);
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
index 0000000,0000000..b422749
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
@@@ -1,0 -1,0 +1,252 @@@
++/*
++ * 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.restrictions;
++
++import java.util.*;
++
++import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.QueryOptions;
++import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction.Contains;
++import org.apache.cassandra.db.IndexExpression;
++import org.apache.cassandra.db.index.SecondaryIndexManager;
++import org.apache.cassandra.exceptions.InvalidRequestException;
++
++/**
++ * Sets of column restrictions.
++ *
++ * <p>This class is immutable in order to be use within {@link PrimaryKeyRestrictionSet} which as
++ * an implementation of {@link Restriction} need to be immutable.
++ */
++final class RestrictionSet implements Restrictions, Iterable<Restriction>
++{
++    /**
++     * The comparator used to sort the <code>Restriction</code>s.
++     */
++    private static final Comparator<ColumnDefinition> COLUMN_DEFINITION_COMPARATOR = new Comparator<ColumnDefinition>()
++    {
++        @Override
++        public int compare(ColumnDefinition column, ColumnDefinition otherColumn)
++        {
++            int value = Integer.compare(column.position(), otherColumn.position());
++            return value != 0 ? value : column.name.bytes.compareTo(otherColumn.name.bytes);
++        }
++    };
++
++    /**
++     * The restrictions per column.
++     */
++    protected final TreeMap<ColumnDefinition, Restriction> restrictions;
++
++    public RestrictionSet()
++    {
++        this(new TreeMap<ColumnDefinition, Restriction>(COLUMN_DEFINITION_COMPARATOR));
++    }
++
++    private RestrictionSet(TreeMap<ColumnDefinition, Restriction> restrictions)
++    {
++        this.restrictions = restrictions;
++    }
++
++    @Override
++    public final void addIndexExpressionTo(List<IndexExpression> expressions,
++                                           SecondaryIndexManager indexManager,
++                                           QueryOptions options) throws InvalidRequestException
++    {
++        for (Restriction restriction : restrictions.values())
++            restriction.addIndexExpressionTo(expressions, indexManager, options);
++    }
++
++    @Override
++    public final Set<ColumnDefinition> getColumnDefs()
++    {
++        return restrictions.keySet();
++    }
++
++    @Override
++    public boolean usesFunction(String ksName, String functionName)
++    {
++        for (Restriction restriction : restrictions.values())
++            if (restriction.usesFunction(ksName, functionName))
++                return true;
++
++        return false;
++    }
++
++    @Override
++    public final boolean isEmpty()
++    {
++        return getColumnDefs().isEmpty();
++    }
++
++    @Override
++    public final int size()
++    {
++        return getColumnDefs().size();
++    }
++
++    /**
++     * Adds the specified restriction to this set of restrictions.
++     *
++     * @param restriction the restriction to add
++     * @return the new set of restrictions
++     * @throws InvalidRequestException if the new restriction cannot be added
++     */
++    public RestrictionSet addRestriction(Restriction restriction) throws InvalidRequestException
++    {
++        // RestrictionSet is immutable so we need to clone the restrictions map.
++        TreeMap<ColumnDefinition, Restriction> newRestrictions = new TreeMap<>(this.restrictions);
++        return new RestrictionSet(mergeRestrictions(newRestrictions, restriction));
++    }
++
++    private TreeMap<ColumnDefinition, Restriction> mergeRestrictions(TreeMap<ColumnDefinition, Restriction> restrictions,
++                                                                     Restriction restriction)
++                                                                     throws InvalidRequestException
++    {
++        Collection<ColumnDefinition> columnDefs = restriction.getColumnDefs();
++        Set<Restriction> existingRestrictions = getRestrictions(columnDefs);
++
++        if (existingRestrictions.isEmpty())
++        {
++            for (ColumnDefinition columnDef : columnDefs)
++                restrictions.put(columnDef, restriction);
++        }
++        else
++        {
++            for (Restriction existing : existingRestrictions)
++            {
++                Restriction newRestriction = mergeRestrictions(existing, restriction);
++
++                for (ColumnDefinition columnDef : columnDefs)
++                    restrictions.put(columnDef, newRestriction);
++            }
++        }
++
++        return restrictions;
++    }
++
++    /**
++     * Returns all the restrictions applied to the specified columns.
++     *
++     * @param columnDefs the column definitions
++     * @return all the restrictions applied to the specified columns
++     */
++    private Set<Restriction> getRestrictions(Collection<ColumnDefinition> columnDefs)
++    {
++        Set<Restriction> set = new HashSet<>();
++        for (ColumnDefinition columnDef : columnDefs)
++        {
++            Restriction existing = restrictions.get(columnDef);
++            if (existing != null)
++                set.add(existing);
++        }
++        return set;
++    }
++
++    @Override
++    public final boolean hasSupportingIndex(SecondaryIndexManager indexManager)
++    {
++        for (Restriction restriction : restrictions.values())
++        {
++            if (restriction.hasSupportingIndex(indexManager))
++                return true;
++        }
++        return false;
++    }
++
++    /**
++     * Returns the column after the specified one.
++     *
++     * @param columnDef the column for which the next one need to be found
++     * @return the column after the specified one.
++     */
++    ColumnDefinition nextColumn(ColumnDefinition columnDef)
++    {
++        return restrictions.tailMap(columnDef, false).firstKey();
++    }
++
++    /**
++     * Returns the definition of the first column.
++     *
++     * @return the definition of the first column.
++     */
++    ColumnDefinition firstColumn()
++    {
++        return isEmpty() ? null : this.restrictions.firstKey();
++    }
++
++    /**
++     * Returns the definition of the last column.
++     *
++     * @return the definition of the last column.
++     */
++    ColumnDefinition lastColumn()
++    {
++        return isEmpty() ? null : this.restrictions.lastKey();
++    }
++
++    /**
++     * Returns the last restriction.
++     *
++     * @return the last restriction.
++     */
++    Restriction lastRestriction()
++    {
++        return isEmpty() ? null : this.restrictions.lastEntry().getValue();
++    }
++
++    /**
++     * Merges the two specified restrictions.
++     *
++     * @param restriction the first restriction
++     * @param otherRestriction the second restriction
++     * @return the merged restriction
++     * @throws InvalidRequestException if the two restrictions cannot be merged
++     */
++    private static Restriction mergeRestrictions(Restriction restriction,
++                                                 Restriction otherRestriction) throws InvalidRequestException
++    {
++        return restriction == null ? otherRestriction
++                                   : restriction.mergeWith(otherRestriction);
++    }
++
++    /**
++     * Checks if the restrictions contains multiple contains, contains key, or map[key] = value.
++     *
++     * @return <code>true</code> if the restrictions contains multiple contains, contains key, or ,
++     * map[key] = value; <code>false</code> otherwise
++     */
++    public final boolean hasMultipleContains()
++    {
++        int numberOfContains = 0;
++        for (Restriction restriction : restrictions.values())
++        {
++            if (restriction.isContains())
++            {
++                Contains contains = (Contains) restriction;
++                numberOfContains += (contains.numberOfValues() + contains.numberOfKeys() + contains.numberOfEntries());
++            }
++        }
++        return numberOfContains > 1;
++    }
++
++    @Override
++    public Iterator<Restriction> iterator()
++    {
++        return new LinkedHashSet<>(restrictions.values()).iterator();
++    }
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
index 3cfe4ab,0000000..e2b31dd
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
@@@ -1,84 -1,0 +1,84 @@@
 +/*
 + * 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.restrictions;
 +
 +import java.util.Collection;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.db.IndexExpression;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * Sets of restrictions
 + */
 +interface Restrictions
 +{
 +    /**
 +     * Returns the column definitions in position order.
 +     * @return the column definitions in position order.
 +     */
 +    public Collection<ColumnDefinition> getColumnDefs();
 +
 +    /**
 +     * Returns <code>true</code> if one of the restrictions use the specified function.
 +     *
 +     * @param ksName the keyspace name
 +     * @param functionName the function name
 +     * @return <code>true</code> if one of the restrictions use the specified function, <code>false</code> otherwise.
 +     */
 +    public boolean usesFunction(String ksName, String functionName);
 +
 +    /**
 +     * Check if the restriction is on indexed columns.
 +     *
 +     * @param indexManager the index manager
 +     * @return <code>true</code> if the restriction is on indexed columns, <code>false</code>
 +     */
 +    public boolean hasSupportingIndex(SecondaryIndexManager indexManager);
 +
 +    /**
 +     * Adds to the specified list the <code>IndexExpression</code>s corresponding to this <code>Restriction</code>.
 +     *
 +     * @param expressions the list to add the <code>IndexExpression</code>s to
 +     * @param indexManager the secondary index manager
 +     * @param options the query options
 +     * @throws InvalidRequestException if this <code>Restriction</code> cannot be converted into
 +     * <code>IndexExpression</code>s
 +     */
 +    public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                     SecondaryIndexManager indexManager,
 +                                     QueryOptions options)
 +                                     throws InvalidRequestException;
 +
 +    /**
-      * Checks if this <code>SingleColumnPrimaryKeyRestrictions</code> is empty or not.
++     * Checks if this <code>PrimaryKeyRestrictionSet</code> is empty or not.
 +     *
-      * @return <code>true</code> if this <code>SingleColumnPrimaryKeyRestrictions</code> is empty, <code>false</code> otherwise.
++     * @return <code>true</code> if this <code>PrimaryKeyRestrictionSet</code> is empty, <code>false</code> otherwise.
 +     */
 +    boolean isEmpty();
 +
 +    /**
 +     * Returns the number of columns that have a restriction.
 +     *
 +     * @return the number of columns that have a restriction.
 +     */
 +    public int size();
 +}


[3/8] 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;
+     }
+ }


[4/8] 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));
          }
      }
  


[5/8] 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/trunk
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/8] 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/trunk
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"})


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

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
index 9fbe462,0000000..56a10a8
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
@@@ -1,521 -1,0 +1,558 @@@
 +/*
 + * 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.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
++import java.util.Collection;
 +import java.util.Collections;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.AbstractMarker;
 +import org.apache.cassandra.cql3.Operator;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
++import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.db.marshal.CompositeType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
 +public abstract class SingleColumnRestriction extends AbstractRestriction
 +{
 +    /**
 +     * The definition of the column to which apply the restriction.
 +     */
 +    protected final ColumnDefinition columnDef;
 +
 +    public SingleColumnRestriction(ColumnDefinition columnDef)
 +    {
 +        this.columnDef = columnDef;
 +    }
 +
-     /**
-      * Returns the definition of the column to which is associated this restriction.
-      * @return the definition of the column to which is associated this restriction
-      */
-     public ColumnDefinition getColumnDef()
++    @Override
++    public Collection<ColumnDefinition> getColumnDefs()
 +    {
-         return columnDef;
++        return Collections.singletonList(columnDef);
 +    }
 +
 +    @Override
-     public void addIndexExpressionTo(List<IndexExpression> expressions,
-                                      SecondaryIndexManager indexManager,
-                                      QueryOptions options) throws InvalidRequestException
++    public ColumnDefinition getFirstColumn()
 +    {
-         List<ByteBuffer> values = values(options);
-         checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns");
++        return columnDef;
++    }
 +
-         ByteBuffer value = validateIndexedValue(columnDef, values.get(0));
-         expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, value));
++    @Override
++    public ColumnDefinition getLastColumn()
++    {
++        return columnDef;
 +    }
 +
 +    @Override
 +    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
 +    {
 +        SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
 +        return index != null && isSupportedBy(index);
 +    }
 +
++    @Override
++    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
++    {
++            checkFalse(otherRestriction.isMultiColumn(),
++                       "Mixing single column relations and multi column relations on clustering columns is not allowed");
++
++            return doMergeWith(otherRestriction);
++    }
++
++    protected abstract Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException;
++
 +    /**
 +     * Check if this type of restriction is supported by the specified index.
 +     *
 +     * @param index the Secondary index
 +     * @return <code>true</code> this type of restriction is supported by the specified index,
 +     * <code>false</code> otherwise.
 +     */
 +    protected abstract boolean isSupportedBy(SecondaryIndex index);
 +
 +    public static final class EQ extends SingleColumnRestriction
 +    {
 +        private final Term value;
 +
 +        public EQ(ColumnDefinition columnDef, Term value)
 +        {
 +            super(columnDef);
 +            this.value = value;
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return usesFunction(value, ksName, functionName);
 +        }
 +
 +        @Override
 +        public boolean isEQ()
 +        {
 +            return true;
 +        }
 +
 +        @Override
-         public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++        public void addIndexExpressionTo(List<IndexExpression> expressions,
++                                         SecondaryIndexManager indexManager,
++                                         QueryOptions options) throws InvalidRequestException
++        {
++            ByteBuffer buffer = validateIndexedValue(columnDef, value.bindAndGet(options));
++            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, buffer));
++        }
++
++        @Override
++        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
-             return Collections.singletonList(value.bindAndGet(options));
++            builder.addElementToAll(value.bindAndGet(options));
++            checkFalse(builder.containsNull(), "Invalid null value in condition for column %s", columnDef);
++            return builder;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("EQ(%s)", value);
 +        }
 +
 +        @Override
-         public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
++        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal", columnDef.name);
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.EQ);
 +        }
 +    }
 +
 +    public static abstract class IN extends SingleColumnRestriction
 +    {
 +        public IN(ColumnDefinition columnDef)
 +        {
 +            super(columnDef);
 +        }
 +
 +        @Override
 +        public final boolean isIN()
 +        {
 +            return true;
 +        }
 +
 +        @Override
-         public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
++        public final Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN", columnDef.name);
 +        }
 +
 +        @Override
++        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
++        {
++            builder.addEachElementToAll(getValues(options));
++            checkFalse(builder.containsNull(), "Invalid null value in condition for column %s", columnDef);
++            return builder;
++        }
++
++        @Override
++        public void addIndexExpressionTo(List<IndexExpression> expressions,
++                                         SecondaryIndexManager indexManager,
++                                         QueryOptions options) throws InvalidRequestException
++        {
++            List<ByteBuffer> values = getValues(options);
++            checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns");
++
++            ByteBuffer value = validateIndexedValue(columnDef, values.get(0));
++            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, value));
++        }
++
++        @Override
 +        protected final boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.IN);
 +        }
++
++        protected abstract List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException;
 +    }
 +
 +    public static class InWithValues extends IN
 +    {
 +        protected final List<Term> values;
 +
 +        public InWithValues(ColumnDefinition columnDef, List<Term> values)
 +        {
 +            super(columnDef);
 +            this.values = values;
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return usesFunction(values, ksName, functionName);
 +        }
 +
 +        @Override
-         public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++        protected List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> buffers = new ArrayList<>(values.size());
 +            for (Term value : values)
 +                buffers.add(value.bindAndGet(options));
 +            return buffers;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("IN(%s)", values);
 +        }
 +    }
 +
 +    public static class InWithMarker extends IN
 +    {
 +        protected final AbstractMarker marker;
 +
 +        public InWithMarker(ColumnDefinition columnDef, AbstractMarker marker)
 +        {
 +            super(columnDef);
 +            this.marker = marker;
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return false;
 +        }
 +
 +        @Override
-         public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++        protected List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            Term.MultiItemTerminal lval = (Term.MultiItemTerminal) marker.bind(options);
 +            if (lval == null)
 +                throw new InvalidRequestException("Invalid null value for IN restriction");
 +            return lval.getElements();
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "IN ?";
 +        }
 +    }
 +
 +    public static class Slice extends SingleColumnRestriction
 +    {
 +        private final TermSlice slice;
 +
 +        public Slice(ColumnDefinition columnDef, Bound bound, boolean inclusive, Term term)
 +        {
 +            super(columnDef);
 +            slice = TermSlice.newInstance(bound, inclusive, term);
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName))
 +                    || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName));
 +        }
 +
 +        @Override
 +        public boolean isSlice()
 +        {
 +            return true;
 +        }
 +
 +        @Override
-         public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            return slice.hasBound(b);
 +        }
 +
 +        @Override
-         public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
++        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
-             return Collections.singletonList(slice.bound(b).bindAndGet(options));
++            return builder.addElementToAll(slice.bound(bound).bindAndGet(options));
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            return slice.isInclusive(b);
 +        }
 +
 +        @Override
-         public Restriction mergeWith(Restriction otherRestriction)
-         throws InvalidRequestException
++        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isSlice(),
 +                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
 +                      columnDef.name);
 +
 +            SingleColumnRestriction.Slice otherSlice = (SingleColumnRestriction.Slice) otherRestriction;
 +
 +            checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
 +                       "More than one restriction was found for the start bound on %s", columnDef.name);
 +
 +            checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
 +                       "More than one restriction was found for the end bound on %s", columnDef.name);
 +
 +            return new Slice(columnDef,  slice.merge(otherSlice.slice));
 +        }
 +
 +        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options) throws InvalidRequestException
 +        {
 +            for (Bound b : Bound.values())
 +            {
 +                if (hasBound(b))
 +                {
 +                    ByteBuffer value = validateIndexedValue(columnDef, slice.bound(b).bindAndGet(options));
 +                    Operator op = slice.getIndexOperator(b);
 +                    // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
 +                    // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
 +                    // use the underlying comparator as is.
 +                    op = columnDef.isReversedType() ? op.reverse() : op;
 +                    expressions.add(new IndexExpression(columnDef.name.bytes, op, value));
 +                }
 +            }
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return slice.isSupportedBy(index);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("SLICE%s", slice);
 +        }
 +
 +        private Slice(ColumnDefinition columnDef, TermSlice slice)
 +        {
 +            super(columnDef);
 +            this.slice = slice;
 +        }
 +    }
 +
 +    // This holds CONTAINS, CONTAINS_KEY, and map[key] = value restrictions because we might want to have any combination of them.
 +    public static final class Contains extends SingleColumnRestriction
 +    {
 +        private List<Term> values = new ArrayList<>(); // for CONTAINS
 +        private List<Term> keys = new ArrayList<>(); // for CONTAINS_KEY
 +        private List<Term> entryKeys = new ArrayList<>(); // for map[key] = value
 +        private List<Term> entryValues = new ArrayList<>(); // for map[key] = value
 +
 +        public Contains(ColumnDefinition columnDef, Term t, boolean isKey)
 +        {
 +            super(columnDef);
 +            if (isKey)
 +                keys.add(t);
 +            else
 +                values.add(t);
 +        }
 +
 +        public Contains(ColumnDefinition columnDef, Term mapKey, Term mapValue)
 +        {
 +            super(columnDef);
 +            entryKeys.add(mapKey);
 +            entryValues.add(mapValue);
 +        }
 +
 +        @Override
-         public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
-             return bindAndGet(values, options);
++            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean isContains()
 +        {
 +            return true;
 +        }
 +
 +        @Override
-         public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
++        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isContains(),
 +                      "Collection column %s can only be restricted by CONTAINS, CONTAINS KEY, or map-entry equality",
-                       getColumnDef().name);
++                      columnDef.name);
 +
-             SingleColumnRestriction.Contains newContains = new Contains(getColumnDef());
++            SingleColumnRestriction.Contains newContains = new Contains(columnDef);
 +
 +            copyKeysAndValues(this, newContains);
 +            copyKeysAndValues((Contains) otherRestriction, newContains);
 +
 +            return newContains;
 +        }
 +
 +        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options)
 +                                         throws InvalidRequestException
 +        {
-             addExpressionsFor(expressions, values(options), Operator.CONTAINS);
-             addExpressionsFor(expressions, keys(options), Operator.CONTAINS_KEY);
++            addExpressionsFor(expressions, bindAndGet(values, options), Operator.CONTAINS);
++            addExpressionsFor(expressions, bindAndGet(keys, options), Operator.CONTAINS_KEY);
 +            addExpressionsFor(expressions, entries(options), Operator.EQ);
 +        }
 +
 +        private void addExpressionsFor(List<IndexExpression> target, List<ByteBuffer> values,
 +                                       Operator op) throws InvalidRequestException
 +        {
 +            for (ByteBuffer value : values)
 +            {
 +                validateIndexedValue(columnDef, value);
 +                target.add(new IndexExpression(columnDef.name.bytes, op, value));
 +            }
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            boolean supported = false;
 +
 +            if (numberOfValues() > 0)
 +                supported |= index.supportsOperator(Operator.CONTAINS);
 +
 +            if (numberOfKeys() > 0)
 +                supported |= index.supportsOperator(Operator.CONTAINS_KEY);
 +
 +            if (numberOfEntries() > 0)
 +                supported |= index.supportsOperator(Operator.EQ);
 +
 +            return supported;
 +        }
 +
 +        public int numberOfValues()
 +        {
 +            return values.size();
 +        }
 +
 +        public int numberOfKeys()
 +        {
 +            return keys.size();
 +        }
 +
 +        public int numberOfEntries()
 +        {
 +            return entryKeys.size();
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return usesFunction(values, ksName, functionName) || usesFunction(keys, ksName, functionName) ||
 +                   usesFunction(entryKeys, ksName, functionName) || usesFunction(entryValues, ksName, functionName);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("CONTAINS(values=%s, keys=%s, entryKeys=%s, entryValues=%s)", values, keys, entryKeys, entryValues);
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
-         public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
++        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
-         private List<ByteBuffer> keys(QueryOptions options) throws InvalidRequestException
-         {
-             return bindAndGet(keys, options);
-         }
- 
 +        private List<ByteBuffer> entries(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> entryBuffers = new ArrayList<>(entryKeys.size());
 +            List<ByteBuffer> keyBuffers = bindAndGet(entryKeys, options);
 +            List<ByteBuffer> valueBuffers = bindAndGet(entryValues, options);
 +            for (int i = 0; i < entryKeys.size(); i++)
 +            {
 +                if (valueBuffers.get(i) == null)
 +                    throw new InvalidRequestException("Unsupported null value for map-entry equality");
 +                entryBuffers.add(CompositeType.build(keyBuffers.get(i), valueBuffers.get(i)));
 +            }
 +            return entryBuffers;
 +        }
 +
 +        /**
 +         * Binds the query options to the specified terms and returns the resulting values.
 +         *
 +         * @param terms the terms
 +         * @param options the query options
 +         * @return the value resulting from binding the query options to the specified terms
 +         * @throws InvalidRequestException if a problem occurs while binding the query options
 +         */
 +        private static List<ByteBuffer> bindAndGet(List<Term> terms, QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> buffers = new ArrayList<>(terms.size());
 +            for (Term value : terms)
 +                buffers.add(value.bindAndGet(options));
 +            return buffers;
 +        }
 +
 +        /**
 +         * Copies the keys and value from the first <code>Contains</code> to the second one.
 +         *
 +         * @param from the <code>Contains</code> to copy from
 +         * @param to the <code>Contains</code> to copy to
 +         */
 +        private static void copyKeysAndValues(Contains from, Contains to)
 +        {
 +            to.values.addAll(from.values);
 +            to.keys.addAll(from.keys);
 +            to.entryKeys.addAll(from.entryKeys);
 +            to.entryValues.addAll(from.entryValues);
 +        }
 +
 +        private Contains(ColumnDefinition columnDef)
 +        {
 +            super(columnDef);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index 403bf6d,0000000..cea1699
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@@ -1,576 -1,0 +1,576 @@@
 +/*
 + * 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.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
 +import com.google.common.base.Joiner;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.Relation;
 +import org.apache.cassandra.cql3.VariableSpecifications;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.IndexExpression;
 +import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.RowPosition;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.dht.*;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.service.StorageService;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
 +/**
 + * The restrictions corresponding to the relations specified on the where-clause of CQL query.
 + */
 +public final class StatementRestrictions
 +{
 +    /**
 +     * The Column Family meta data
 +     */
 +    public final CFMetaData cfm;
 +
 +    /**
 +     * Restrictions on partitioning columns
 +     */
 +    private PrimaryKeyRestrictions partitionKeyRestrictions;
 +
 +    /**
 +     * Restrictions on clustering columns
 +     */
 +    private PrimaryKeyRestrictions clusteringColumnsRestrictions;
 +
 +    /**
 +     * Restriction on non-primary key columns (i.e. secondary index restrictions)
 +     */
-     private SingleColumnRestrictions nonPrimaryKeyRestrictions;
++    private RestrictionSet nonPrimaryKeyRestrictions;
 +
 +    /**
 +     * The restrictions used to build the index expressions
 +     */
 +    private final List<Restrictions> indexRestrictions = new ArrayList<>();
 +
 +    /**
 +     * <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise
 +     */
 +    private boolean usesSecondaryIndexing;
 +
 +    /**
 +     * Specify if the query will return a range of partition keys.
 +     */
 +    private boolean isKeyRange;
 +
 +    /**
 +     * Creates a new empty <code>StatementRestrictions</code>.
 +     *
 +     * @param cfm the column family meta data
 +     * @return a new empty <code>StatementRestrictions</code>.
 +     */
 +    public static StatementRestrictions empty(CFMetaData cfm)
 +    {
 +        return new StatementRestrictions(cfm);
 +    }
 +
 +    private StatementRestrictions(CFMetaData cfm)
 +    {
 +        this.cfm = cfm;
-         this.partitionKeyRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.getKeyValidatorAsCType());
-         this.clusteringColumnsRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.comparator);
-         this.nonPrimaryKeyRestrictions = new SingleColumnRestrictions();
++        this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
++        this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
++        this.nonPrimaryKeyRestrictions = new RestrictionSet();
 +    }
 +
 +    public StatementRestrictions(CFMetaData cfm,
 +            List<Relation> whereClause,
 +            VariableSpecifications boundNames,
 +            boolean selectsOnlyStaticColumns,
 +            boolean selectACollection) throws InvalidRequestException
 +    {
 +        this.cfm = cfm;
-         this.partitionKeyRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.getKeyValidatorAsCType());
-         this.clusteringColumnsRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.comparator);
-         this.nonPrimaryKeyRestrictions = new SingleColumnRestrictions();
++        this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
++        this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
++        this.nonPrimaryKeyRestrictions = new RestrictionSet();
 +
 +        /*
 +         * WHERE clause. For a given entity, rules are: - EQ relation conflicts with anything else (including a 2nd EQ)
 +         * - Can't have more than one LT(E) relation (resp. GT(E) relation) - IN relation are restricted to row keys
 +         * (for now) and conflicts with anything else (we could allow two IN for the same entity but that doesn't seem
 +         * very useful) - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value
 +         * in CQL so far)
 +         */
 +        for (Relation relation : whereClause)
 +            addRestriction(relation.toRestriction(cfm, boundNames));
 +
 +        ColumnFamilyStore cfs = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
 +        SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
 +
 +        boolean hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(secondaryIndexManager);
 +        boolean hasQueriableIndex = hasQueriableClusteringColumnIndex
 +                || partitionKeyRestrictions.hasSupportingIndex(secondaryIndexManager)
 +                || nonPrimaryKeyRestrictions.hasSupportingIndex(secondaryIndexManager);
 +
 +        // At this point, the select statement if fully constructed, but we still have a few things to validate
 +        processPartitionKeyRestrictions(hasQueriableIndex);
 +
 +        // Some but not all of the partition key columns have been specified;
 +        // hence we need turn these restrictions into index expressions.
 +        if (usesSecondaryIndexing)
 +            indexRestrictions.add(partitionKeyRestrictions);
 +
 +        checkFalse(selectsOnlyStaticColumns && hasClusteringColumnsRestriction(),
 +                   "Cannot restrict clustering columns when selecting only static columns");
 +
 +        processClusteringColumnsRestrictions(hasQueriableIndex, selectACollection);
 +
 +        // Covers indexes on the first clustering column (among others).
 +        if (isKeyRange && hasQueriableClusteringColumnIndex)
 +            usesSecondaryIndexing = true;
 +
 +        usesSecondaryIndexing = usesSecondaryIndexing || clusteringColumnsRestrictions.isContains();
 +
 +        if (usesSecondaryIndexing)
 +            indexRestrictions.add(clusteringColumnsRestrictions);
 +
 +        // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
 +        // there is restrictions not covered by the PK.
 +        if (!nonPrimaryKeyRestrictions.isEmpty())
 +        {
 +            usesSecondaryIndexing = true;
 +            indexRestrictions.add(nonPrimaryKeyRestrictions);
 +        }
 +
 +        if (usesSecondaryIndexing)
 +            validateSecondaryIndexSelections(selectsOnlyStaticColumns);
 +    }
 +
 +    private void addRestriction(Restriction restriction) throws InvalidRequestException
 +    {
 +        if (restriction.isMultiColumn())
 +            clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
 +        else if (restriction.isOnToken())
 +            partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
 +        else
 +            addSingleColumnRestriction((SingleColumnRestriction) restriction);
 +    }
 +
 +    public boolean usesFunction(String ksName, String functionName)
 +    {
 +        return  partitionKeyRestrictions.usesFunction(ksName, functionName)
 +                || clusteringColumnsRestrictions.usesFunction(ksName, functionName)
 +                || nonPrimaryKeyRestrictions.usesFunction(ksName, functionName);
 +    }
 +
 +    private void addSingleColumnRestriction(SingleColumnRestriction restriction) throws InvalidRequestException
 +    {
-         ColumnDefinition def = restriction.getColumnDef();
++        ColumnDefinition def = restriction.columnDef;
 +        if (def.isPartitionKey())
 +            partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
 +        else if (def.isClusteringColumn())
 +            clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
 +        else
 +            nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction(restriction);
 +    }
 +
 +    /**
 +     * Checks if the restrictions on the partition key is an IN restriction.
 +     *
 +     * @return <code>true</code> the restrictions on the partition key is an IN restriction, <code>false</code>
 +     * otherwise.
 +     */
 +    public boolean keyIsInRelation()
 +    {
 +        return partitionKeyRestrictions.isIN();
 +    }
 +
 +    /**
 +     * Checks if the query request a range of partition keys.
 +     *
 +     * @return <code>true</code> if the query request a range of partition keys, <code>false</code> otherwise.
 +     */
 +    public boolean isKeyRange()
 +    {
 +        return this.isKeyRange;
 +    }
 +
 +    /**
 +     * Checks if the secondary index need to be queried.
 +     *
 +     * @return <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise.
 +     */
 +    public boolean usesSecondaryIndexing()
 +    {
 +        return this.usesSecondaryIndexing;
 +    }
 +
 +    private void processPartitionKeyRestrictions(boolean hasQueriableIndex) throws InvalidRequestException
 +    {
 +        // If there is a queriable index, no special condition are required on the other restrictions.
 +        // But we still need to know 2 things:
 +        // - If we don't have a queriable index, is the query ok
 +        // - Is it queriable without 2ndary index, which is always more efficient
 +        // If a component of the partition key is restricted by a relation, all preceding
 +        // components must have a EQ. Only the last partition key component can be in IN relation.
 +        if (partitionKeyRestrictions.isOnToken())
 +            isKeyRange = true;
 +
 +        if (hasPartitionKeyUnrestrictedComponents())
 +        {
 +            if (!partitionKeyRestrictions.isEmpty())
 +            {
 +                if (!hasQueriableIndex)
 +                    throw invalidRequest("Partition key parts: %s must be restricted as other parts are",
 +                                         Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents()));
 +            }
 +
 +            isKeyRange = true;
 +            usesSecondaryIndexing = hasQueriableIndex;
 +        }
 +    }
 +
 +    /**
 +     * Checks if the partition key has some unrestricted components.
 +     * @return <code>true</code> if the partition key has some unrestricted components, <code>false</code> otherwise.
 +     */
 +    private boolean hasPartitionKeyUnrestrictedComponents()
 +    {
 +        return partitionKeyRestrictions.size() <  cfm.partitionKeyColumns().size();
 +    }
 +
 +    /**
 +     * Returns the partition key components that are not restricted.
 +     * @return the partition key components that are not restricted.
 +     */
 +    private List<ColumnIdentifier> getPartitionKeyUnrestrictedComponents()
 +    {
 +        List<ColumnDefinition> list = new ArrayList<>(cfm.partitionKeyColumns());
 +        list.removeAll(partitionKeyRestrictions.getColumnDefs());
 +        return ColumnDefinition.toIdentifiers(list);
 +    }
 +
 +    /**
 +     * Processes the clustering column restrictions.
 +     *
 +     * @param hasQueriableIndex <code>true</code> if some of the queried data are indexed, <code>false</code> otherwise
 +     * @param selectACollection <code>true</code> if the query should return a collection column
 +     * @throws InvalidRequestException if the request is invalid
 +     */
 +    private void processClusteringColumnsRestrictions(boolean hasQueriableIndex,
 +                                                      boolean selectACollection) throws InvalidRequestException
 +    {
 +        checkFalse(clusteringColumnsRestrictions.isIN() && selectACollection,
 +                   "Cannot restrict clustering columns by IN relations when a collection is selected by the query");
 +        checkFalse(clusteringColumnsRestrictions.isContains() && !hasQueriableIndex,
 +                   "Cannot restrict clustering columns by a CONTAINS relation without a secondary index");
 +
 +        if (hasClusteringColumnsRestriction())
 +        {
 +            List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
 +            List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
 +
 +            for (int i = 0, m = restrictedColumns.size(); i < m; i++)
 +            {
 +                ColumnDefinition clusteringColumn = clusteringColumns.get(i);
 +                ColumnDefinition restrictedColumn = restrictedColumns.get(i);
 +
 +                if (!clusteringColumn.equals(restrictedColumn))
 +                {
 +                    checkTrue(hasQueriableIndex,
 +                              "PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted",
 +                              restrictedColumn.name,
 +                              clusteringColumn.name);
 +
 +                    usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
 +                    break;
 +                }
 +            }
 +        }
 +
 +        if (clusteringColumnsRestrictions.isContains())
 +            usesSecondaryIndexing = true;
 +    }
 +
 +    public List<IndexExpression> getIndexExpressions(SecondaryIndexManager indexManager,
 +                                                     QueryOptions options) throws InvalidRequestException
 +    {
 +        if (!usesSecondaryIndexing || indexRestrictions.isEmpty())
 +            return Collections.emptyList();
 +
 +        List<IndexExpression> expressions = new ArrayList<>();
 +        for (Restrictions restrictions : indexRestrictions)
 +            restrictions.addIndexExpressionTo(expressions, indexManager, options);
 +
 +        return expressions;
 +    }
 +
 +    /**
 +     * Returns the partition keys for which the data is requested.
 +     *
 +     * @param options the query options
 +     * @return the partition keys for which the data is requested.
 +     * @throws InvalidRequestException if the partition keys cannot be retrieved
 +     */
 +    public Collection<ByteBuffer> getPartitionKeys(final QueryOptions options) throws InvalidRequestException
 +    {
 +        return partitionKeyRestrictions.values(options);
 +    }
 +
 +    /**
 +     * Returns the specified bound of the partition key.
 +     *
 +     * @param b the boundary type
 +     * @param options the query options
 +     * @return the specified bound of the partition key
 +     * @throws InvalidRequestException if the boundary cannot be retrieved
 +     */
 +    private ByteBuffer getPartitionKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
 +    {
 +        // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the
 +        // first
 +        // component of a composite partition key).
 +        if (hasPartitionKeyUnrestrictedComponents())
 +            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 partitionKeyRestrictions.bounds(b, options).get(0);
 +    }
 +
 +    /**
 +     * Returns the partition key bounds.
 +     *
 +     * @param options the query options
 +     * @return the partition key bounds
 +     * @throws InvalidRequestException if the query is invalid
 +     */
 +    public AbstractBounds<RowPosition> getPartitionKeyBounds(QueryOptions options) throws InvalidRequestException
 +    {
 +        IPartitioner p = StorageService.getPartitioner();
 +
 +        if (partitionKeyRestrictions.isOnToken())
 +        {
 +            return getPartitionKeyBoundsForTokenRestrictions(p, options);
 +        }
 +
 +        return getPartitionKeyBounds(p, options);
 +    }
 +
 +    private AbstractBounds<RowPosition> getPartitionKeyBounds(IPartitioner p,
 +                                                              QueryOptions options) throws InvalidRequestException
 +    {
 +        ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options);
 +        ByteBuffer finishKeyBytes = getPartitionKeyBound(Bound.END, options);
 +
 +        RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
 +        RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
 +
 +        if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum())
 +            return null;
 +
 +        if (partitionKeyRestrictions.isInclusive(Bound.START))
 +        {
 +            return partitionKeyRestrictions.isInclusive(Bound.END)
 +                    ? new Bounds<>(startKey, finishKey)
 +                    : new IncludingExcludingBounds<>(startKey, finishKey);
 +        }
 +
 +        return partitionKeyRestrictions.isInclusive(Bound.END)
 +                ? new Range<>(startKey, finishKey)
 +                : new ExcludingBounds<>(startKey, finishKey);
 +    }
 +
 +    private AbstractBounds<RowPosition> getPartitionKeyBoundsForTokenRestrictions(IPartitioner p,
 +                                                                                  QueryOptions options)
 +                                                                                          throws InvalidRequestException
 +    {
 +        Token startToken = getTokenBound(Bound.START, options, p);
 +        Token endToken = getTokenBound(Bound.END, options, p);
 +
 +        boolean includeStart = partitionKeyRestrictions.isInclusive(Bound.START);
 +        boolean includeEnd = partitionKeyRestrictions.isInclusive(Bound.END);
 +
 +        /*
 +         * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
 +         * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result in that
 +         * case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
 +         *
 +         * In practice, we want to return an empty result set if either startToken > endToken, or both are equal but
 +         * one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a) or (a, a)).
 +         * Note though that in the case where startToken or endToken is the minimum token, then this special case
 +         * rule should not apply.
 +         */
 +        int cmp = startToken.compareTo(endToken);
 +        if (!startToken.isMinimum() && !endToken.isMinimum()
 +                && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
 +            return null;
 +
 +        RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
 +        RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
 +
 +        return new Range<>(start, end);
 +    }
 +
 +    private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
 +    {
 +        if (!partitionKeyRestrictions.hasBound(b))
 +            return p.getMinimumToken();
 +
 +        ByteBuffer value = partitionKeyRestrictions.bounds(b, options).get(0);
 +        checkNotNull(value, "Invalid null token value");
 +        return p.getTokenFactory().fromByteArray(value);
 +    }
 +
 +    /**
 +     * Checks if the query does not contains any restriction on the clustering columns.
 +     *
 +     * @return <code>true</code> if the query does not contains any restriction on the clustering columns,
 +     * <code>false</code> otherwise.
 +     */
 +    public boolean hasNoClusteringColumnsRestriction()
 +    {
 +        return clusteringColumnsRestrictions.isEmpty();
 +    }
 +
 +    // For non-composite slices, we don't support internally the difference between exclusive and
 +    // inclusive bounds, so we deal with it manually.
 +    public boolean isNonCompositeSliceWithExclusiveBounds()
 +    {
 +        return !cfm.comparator.isCompound()
 +                && clusteringColumnsRestrictions.isSlice()
 +                && (!clusteringColumnsRestrictions.isInclusive(Bound.START) || !clusteringColumnsRestrictions.isInclusive(Bound.END));
 +    }
 +
 +    /**
 +     * Returns the requested clustering columns as <code>Composite</code>s.
 +     *
 +     * @param options the query options
 +     * @return the requested clustering columns as <code>Composite</code>s
 +     * @throws InvalidRequestException if the query is not valid
 +     */
 +    public List<Composite> getClusteringColumnsAsComposites(QueryOptions options) throws InvalidRequestException
 +    {
 +        return clusteringColumnsRestrictions.valuesAsComposites(options);
 +    }
 +
 +    /**
 +     * Returns the bounds (start or end) of the clustering columns as <code>Composites</code>.
 +     *
 +     * @param b the bound type
 +     * @param options the query options
 +     * @return the bounds (start or end) of the clustering columns as <code>Composites</code>
 +     * @throws InvalidRequestException if the request is not valid
 +     */
 +    public List<Composite> getClusteringColumnsBoundsAsComposites(Bound b,
 +                                                                  QueryOptions options) throws InvalidRequestException
 +    {
 +        return clusteringColumnsRestrictions.boundsAsComposites(b, options);
 +    }
 +
 +    /**
 +     * Returns the bounds (start or end) of the clustering columns.
 +     *
 +     * @param b the bound type
 +     * @param options the query options
 +     * @return the bounds (start or end) of the clustering columns
 +     * @throws InvalidRequestException if the request is not valid
 +     */
 +    public List<ByteBuffer> getClusteringColumnsBounds(Bound b, QueryOptions options) throws InvalidRequestException
 +    {
 +        return clusteringColumnsRestrictions.bounds(b, options);
 +    }
 +
 +    /**
 +     * Checks if the bounds (start or end) of the clustering columns are inclusive.
 +     *
 +     * @param bound the bound type
 +     * @return <code>true</code> if the bounds (start or end) of the clustering columns are inclusive,
 +     * <code>false</code> otherwise
 +     */
 +    public boolean areRequestedBoundsInclusive(Bound bound)
 +    {
 +        return clusteringColumnsRestrictions.isInclusive(bound);
 +    }
 +
 +    /**
 +     * Checks if the query returns a range of columns.
 +     *
 +     * @return <code>true</code> if the query returns a range of columns, <code>false</code> otherwise.
 +     */
 +    public boolean isColumnRange()
 +    {
 +        // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
 +        // Static CF (non dense but non composite) never entails a column slice however
 +        if (!cfm.comparator.isDense())
 +            return cfm.comparator.isCompound();
 +
 +        // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about
 +        // CASSANDRA-5762),
 +        // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
 +        return clusteringColumnsRestrictions.size() < cfm.clusteringColumns().size() || clusteringColumnsRestrictions.isSlice();
 +    }
 +
 +    /**
 +     * Checks if the query need to use filtering.
 +     * @return <code>true</code> if the query need to use filtering, <code>false</code> otherwise.
 +     */
 +    public boolean needFiltering()
 +    {
 +        int numberOfRestrictedColumns = 0;
 +        for (Restrictions restrictions : indexRestrictions)
 +            numberOfRestrictedColumns += restrictions.size();
 +
 +        return numberOfRestrictedColumns > 1
 +                || (numberOfRestrictedColumns == 0 && !clusteringColumnsRestrictions.isEmpty())
 +                || (numberOfRestrictedColumns != 0
 +                        && nonPrimaryKeyRestrictions.hasMultipleContains());
 +    }
 +
 +    private void validateSecondaryIndexSelections(boolean selectsOnlyStaticColumns) throws InvalidRequestException
 +    {
 +        checkFalse(keyIsInRelation(),
 +                   "Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
 +        // When the user only select static columns, the intent is that we don't query the whole partition but just
 +        // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on
 +        // static columns
 +        // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
 +        checkFalse(selectsOnlyStaticColumns, "Queries using 2ndary indexes don't support selecting only static columns");
 +    }
 +
 +    /**
 +     * Checks if the query has some restrictions on the clustering columns.
 +     *
 +     * @return <code>true</code> if the query has some restrictions on the clustering columns,
 +     * <code>false</code> otherwise.
 +     */
 +    private boolean hasClusteringColumnsRestriction()
 +    {
 +        return !clusteringColumnsRestrictions.isEmpty();
 +    }
 +
 +    public void reverse()
 +    {
 +        clusteringColumnsRestrictions = new ReversedPrimaryKeyRestrictions(clusteringColumnsRestrictions);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
index cbcec07,0000000..5848c91
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
@@@ -1,255 -1,0 +1,274 @@@
 +/*
 + * 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.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.List;
 +
 +import com.google.common.base.Joiner;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
 +import org.apache.cassandra.db.composites.CType;
 +import org.apache.cassandra.db.composites.Composite;
++import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
 +/**
 + * <code>Restriction</code> using the token function.
 + */
 +public abstract class TokenRestriction extends AbstractPrimaryKeyRestrictions
 +{
 +    /**
 +     * The definition of the columns to which apply the token restriction.
 +     */
 +    protected final List<ColumnDefinition> columnDefs;
 +
 +    /**
 +     * Creates a new <code>TokenRestriction</code> that apply to the specified columns.
 +     *
 +     * @param ctype the composite type
 +     * @param columnDefs the definition of the columns to which apply the token restriction
 +     */
 +    public TokenRestriction(CType ctype, List<ColumnDefinition> columnDefs)
 +    {
 +        super(ctype);
 +        this.columnDefs = columnDefs;
 +    }
 +
 +    @Override
 +    public  boolean isOnToken()
 +    {
 +        return true;
 +    }
 +
 +    @Override
 +    public Collection<ColumnDefinition> getColumnDefs()
 +    {
 +        return columnDefs;
 +    }
 +
 +    @Override
++    public ColumnDefinition getFirstColumn()
++    {
++        return columnDefs.get(0);
++    }
++
++    @Override
++    public ColumnDefinition getLastColumn()
++    {
++        return columnDefs.get(columnDefs.size() - 1);
++    }
++
++    @Override
 +    public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager)
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public final void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                     SecondaryIndexManager indexManager,
 +                                     QueryOptions options)
 +    {
 +        throw new UnsupportedOperationException("Index expression cannot be created for token restriction");
 +    }
 +
 +    @Override
++    public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
++    {
++        throw new UnsupportedOperationException();
++    }
++
++    @Override
 +    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    @Override
 +    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    /**
 +     * Returns the column names as a comma separated <code>String</code>.
 +     *
 +     * @return the column names as a comma separated <code>String</code>.
 +     */
 +    protected final String getColumnNamesAsString()
 +    {
 +        return Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs));
 +    }
 +
 +    @Override
 +    public final PrimaryKeyRestrictions mergeWith(Restriction otherRestriction) throws InvalidRequestException
 +    {
 +        if (!otherRestriction.isOnToken())
 +            return new TokenFilter(toPrimaryKeyRestriction(otherRestriction), this);
 +
 +        return doMergeWith((TokenRestriction) otherRestriction);
 +    }
 +
 +    /**
 +     * Merges this restriction with the specified <code>TokenRestriction</code>.
 +     * @param otherRestriction the <code>TokenRestriction</code> to merge with.
 +     */
 +    protected abstract PrimaryKeyRestrictions doMergeWith(TokenRestriction otherRestriction) throws InvalidRequestException;
 +
 +    /**
 +     * Converts the specified restriction into a <code>PrimaryKeyRestrictions</code>.
 +     *
 +     * @param restriction the restriction to convert
 +     * @return a <code>PrimaryKeyRestrictions</code>
 +     * @throws InvalidRequestException if a problem occurs while converting the restriction
 +     */
 +    private PrimaryKeyRestrictions toPrimaryKeyRestriction(Restriction restriction) throws InvalidRequestException
 +    {
 +        if (restriction instanceof PrimaryKeyRestrictions)
 +            return (PrimaryKeyRestrictions) restriction;
 +
-         return new SingleColumnPrimaryKeyRestrictions(ctype).mergeWith(restriction);
++        return new PrimaryKeyRestrictionSet(ctype).mergeWith(restriction);
 +    }
 +
 +    public static final class EQ extends TokenRestriction
 +    {
 +        private final Term value;
 +
 +        public EQ(CType ctype, List<ColumnDefinition> columnDefs, Term value)
 +        {
 +            super(ctype, columnDefs);
 +            this.value = value;
 +        }
 +
 +        @Override
 +        public boolean isEQ()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return usesFunction(value, ksName, functionName);
 +        }
 +
 +        @Override
 +        protected PrimaryKeyRestrictions doMergeWith(TokenRestriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
 +                                 Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs)));
 +        }
 +
 +        @Override
 +        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
 +        {
 +            return Collections.singletonList(value.bindAndGet(options));
 +        }
 +    }
 +
 +    public static class Slice extends TokenRestriction
 +    {
 +        private final TermSlice slice;
 +
 +        public Slice(CType ctype, List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
 +        {
 +            super(ctype, columnDefs);
 +            slice = TermSlice.newInstance(bound, inclusive, term);
 +        }
 +
 +        @Override
 +        public boolean isSlice()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            return slice.hasBound(b);
 +        }
 +
 +        @Override
 +        public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
 +        {
 +            return Collections.singletonList(slice.bound(b).bindAndGet(options));
 +        }
 +
 +        @Override
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName))
 +                    || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName));
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            return slice.isInclusive(b);
 +        }
 +
 +        @Override
 +        protected PrimaryKeyRestrictions doMergeWith(TokenRestriction otherRestriction)
 +        throws InvalidRequestException
 +        {
 +            if (!otherRestriction.isSlice())
 +                throw invalidRequest("Columns \"%s\" cannot be restricted by both an equality and an inequality relation",
 +                                     getColumnNamesAsString());
 +
 +            TokenRestriction.Slice otherSlice = (TokenRestriction.Slice) otherRestriction;
 +
 +            if (hasBound(Bound.START) && otherSlice.hasBound(Bound.START))
 +                throw invalidRequest("More than one restriction was found for the start bound on %s",
 +                                     getColumnNamesAsString());
 +
 +            if (hasBound(Bound.END) && otherSlice.hasBound(Bound.END))
 +                throw invalidRequest("More than one restriction was found for the end bound on %s",
 +                                     getColumnNamesAsString());
 +
 +            return new Slice(ctype, columnDefs,  slice.merge(otherSlice.slice));
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("SLICE%s", slice);
 +        }
 +
 +        private Slice(CType ctype, List<ColumnDefinition> columnDefs, TermSlice slice)
 +        {
 +            super(ctype, columnDefs);
 +            this.slice = slice;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 8347ef5,60558b4..683ed49
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -22,17 -22,14 +22,19 @@@ import java.util.*
  
  import com.google.common.base.Function;
  import com.google.common.collect.Iterables;
++import com.google.common.collect.Lists;
  
  import org.apache.cassandra.auth.Permission;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.ColumnDefinition;
  import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.restrictions.Restriction;
 +import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction;
 +import org.apache.cassandra.cql3.selection.Selection;
  import org.apache.cassandra.db.*;
--import org.apache.cassandra.db.composites.CBuilder;
  import org.apache.cassandra.db.composites.Composite;
++import org.apache.cassandra.db.composites.Composites;
++import org.apache.cassandra.db.composites.CompositesBuilder;
  import org.apache.cassandra.db.filter.ColumnSlice;
  import org.apache.cassandra.db.filter.SliceQueryFilter;
  import org.apache.cassandra.db.marshal.BooleanType;
@@@ -44,6 -41,6 +46,12 @@@ import org.apache.cassandra.thrift.Thri
  import org.apache.cassandra.transport.messages.ResultMessage;
  import org.apache.cassandra.utils.Pair;
  
++import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
++
++import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
++
++import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
++
  /*
   * Abstract parent class of individual modifications, i.e. INSERT, UPDATE and DELETE.
   */
@@@ -287,38 -287,38 +295,23 @@@ public abstract class ModificationState
      public List<ByteBuffer> buildPartitionKeyNames(QueryOptions options)
      throws InvalidRequestException
      {
--        CBuilder keyBuilder = cfm.getKeyValidatorAsCType().builder();
--        List<ByteBuffer> keys = new ArrayList<ByteBuffer>();
++        CompositesBuilder keyBuilder = new CompositesBuilder(cfm.getKeyValidatorAsCType());
          for (ColumnDefinition def : cfm.partitionKeyColumns())
          {
--            Restriction r = processedKeys.get(def.name);
--            if (r == null)
--                throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", def.name));
--
--            List<ByteBuffer> values = r.values(options);
++            Restriction r = checkNotNull(processedKeys.get(def.name), "Missing mandatory PRIMARY KEY part %s", def.name);
++            r.appendTo(keyBuilder, options);
++        }
  
--            if (keyBuilder.remainingCount() == 1)
--            {
--                for (ByteBuffer val : values)
--                {
--                    if (val == null)
--                        throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
--                    ByteBuffer key = keyBuilder.buildWith(val).toByteBuffer();
--                    ThriftValidation.validateKey(cfm, key);
--                    keys.add(key);
--                }
--            }
--            else
++        return Lists.transform(keyBuilder.build(), new Function<Composite, ByteBuffer>()
++        {
++            @Override
++            public ByteBuffer apply(Composite composite)
              {
--                if (values.size() != 1)
--                    throw new InvalidRequestException("IN is only supported on the last column of the partition key");
--                ByteBuffer val = values.get(0);
--                if (val == null)
--                    throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
--                keyBuilder.add(val);
++                ByteBuffer byteBuffer = composite.toByteBuffer();
++                ThriftValidation.validateKey(cfm, byteBuffer);
++                return byteBuffer;
              }
--        }
--        return keys;
++        });
      }
  
      public Composite createClusteringPrefix(QueryOptions options)
@@@ -359,7 -359,7 +352,7 @@@
      private Composite createClusteringPrefixBuilderInternal(QueryOptions options)
      throws InvalidRequestException
      {
--        CBuilder builder = cfm.comparator.prefixBuilder();
++        CompositesBuilder builder = new CompositesBuilder(cfm.comparator);
          ColumnDefinition firstEmptyKey = null;
          for (ColumnDefinition def : cfm.clusteringColumns())
          {
@@@ -367,24 -367,24 +360,19 @@@
              if (r == null)
              {
                  firstEmptyKey = def;
--                if (requireFullClusteringKey() && !cfm.comparator.isDense() && cfm.comparator.isCompound())
--                    throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", def.name));
++                checkFalse(requireFullClusteringKey() && !cfm.comparator.isDense() && cfm.comparator.isCompound(), 
++                           "Missing mandatory PRIMARY KEY part %s", def.name);
              }
              else if (firstEmptyKey != null)
              {
--                throw new InvalidRequestException(String.format("Missing PRIMARY KEY part %s since %s is set", firstEmptyKey.name, def.name));
++                throw invalidRequest("Missing PRIMARY KEY part %s since %s is set", firstEmptyKey.name, def.name);
              }
              else
              {
--                List<ByteBuffer> values = r.values(options);
--                assert values.size() == 1; // We only allow IN for row keys so far
--                ByteBuffer val = values.get(0);
--                if (val == null)
--                    throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name));
--                builder.add(val);
++                r.appendTo(builder, options);
              }
          }
--        return builder.build();
++        return builder.build().get(0); // We only allow IN for row keys so far
      }
  
      protected ColumnDefinition getFirstEmptyKey()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71778eec/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
index 9a32dcc,0000000..48bc802
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
+++ b/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
@@@ -1,271 -1,0 +1,292 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db.composites;
 +
 +import java.nio.ByteBuffer;
- import java.util.ArrayList;
- import java.util.Comparator;
- import java.util.LinkedHashSet;
- import java.util.List;
- import java.util.Set;
- import java.util.TreeSet;
++import java.util.*;
 +
 +import org.apache.cassandra.db.composites.Composite.EOC;
 +
 +import static java.util.Collections.singletonList;
 +
 +/**
 + * Builder that allow to build multiple composites at the same time.
 + */
 +public final class CompositesBuilder
 +{
 +    /**
-      * The builder used to build the <code>Composite</code>s.
++     * The composite type.
 +     */
-     private final CBuilder builder;
- 
-     /**
-      * The comparator used to sort the returned <code>Composite</code>s.
-      */
-     private final Comparator<Composite> comparator;
++    private final CType ctype;
 +
 +    /**
 +     * The elements of the composites
 +     */
 +    private final List<List<ByteBuffer>> elementsList = new ArrayList<>();
 +
 +    /**
-      * The number of elements that still can be added.
++     * The number of elements that have been added.
 +     */
-     private int remaining;
++    private int size;
 +
 +    /**
 +     * <code>true</code> if the composites have been build, <code>false</code> otherwise.
 +     */
 +    private boolean built;
 +
 +    /**
 +     * <code>true</code> if the composites contains some <code>null</code> elements.
 +     */
 +    private boolean containsNull;
 +
-     public CompositesBuilder(CBuilder builder, Comparator<Composite> comparator)
++    /**
++     * <code>true</code> if some empty collection have been added.
++     */
++    private boolean hasMissingElements;
++
++    public CompositesBuilder(CType ctype)
 +    {
-         this.builder = builder;
-         this.comparator = comparator;
-         this.remaining = builder.remainingCount();
++        this.ctype = ctype;
 +    }
 +
 +    /**
 +     * Adds the specified element to all the composites.
 +     * <p>
 +     * If this builder contains 2 composites: A-B and A-C a call to this method to add D will result in the composites:
 +     * A-B-D and A-C-D.
 +     * </p>
 +     *
 +     * @param value the value of the next element
 +     * @return this <code>CompositeBuilder</code>
 +     */
 +    public CompositesBuilder addElementToAll(ByteBuffer value)
 +    {
 +        checkUpdateable();
 +
 +        if (isEmpty())
 +            elementsList.add(new ArrayList<ByteBuffer>());
 +
 +        for (int i = 0, m = elementsList.size(); i < m; i++)
 +        {
 +            if (value == null)
 +                containsNull = true;
 +
 +            elementsList.get(i).add(value);
 +        }
-         remaining--;
++        size++;
 +        return this;
 +    }
 +
 +    /**
 +     * Adds individually each of the specified elements to the end of all of the existing composites.
 +     * <p>
 +     * If this builder contains 2 composites: A-B and A-C a call to this method to add D and E will result in the 4
 +     * composites: A-B-D, A-B-E, A-C-D and A-C-E.
 +     * </p>
 +     *
 +     * @param values the elements to add
 +     * @return this <code>CompositeBuilder</code>
 +     */
 +    public CompositesBuilder addEachElementToAll(List<ByteBuffer> values)
 +    {
 +        checkUpdateable();
 +
 +        if (isEmpty())
 +            elementsList.add(new ArrayList<ByteBuffer>());
 +
-         for (int i = 0, m = elementsList.size(); i < m; i++)
++        if (values.isEmpty())
 +        {
-             List<ByteBuffer> oldComposite = elementsList.remove(0);
- 
-             for (int j = 0, n = values.size(); j < n; j++)
++            hasMissingElements = true;
++        }
++        else
++        {
++            for (int i = 0, m = elementsList.size(); i < m; i++)
 +            {
-                 List<ByteBuffer> newComposite = new ArrayList<>(oldComposite);
-                 elementsList.add(newComposite);
++                List<ByteBuffer> oldComposite = elementsList.remove(0);
 +
-                 ByteBuffer value = values.get(j);
++                for (int j = 0, n = values.size(); j < n; j++)
++                {
++                    List<ByteBuffer> newComposite = new ArrayList<>(oldComposite);
++                    elementsList.add(newComposite);
 +
-                 if (value == null)
-                     containsNull = true;
++                    ByteBuffer value = values.get(j);
 +
-                 newComposite.add(values.get(j));
++                    if (value == null)
++                        containsNull = true;
++
++                    newComposite.add(values.get(j));
++                }
 +            }
 +        }
- 
-         remaining--;
++        size++;
 +        return this;
 +    }
 +
 +
 +    /**
 +     * Adds individually each of the specified list of elements to the end of all of the existing composites.
 +     * <p>
 +     * If this builder contains 2 composites: A-B and A-C a call to this method to add [[D, E], [F, G]] will result in the 4
 +     * composites: A-B-D-E, A-B-F-G, A-C-D-E and A-C-F-G.
 +     * </p>
 +     *
 +     * @param values the elements to add
 +     * @return this <code>CompositeBuilder</code>
 +     */
 +    public CompositesBuilder addAllElementsToAll(List<List<ByteBuffer>> values)
 +    {
-         assert !values.isEmpty();
 +        checkUpdateable();
 +
 +        if (isEmpty())
 +            elementsList.add(new ArrayList<ByteBuffer>());
 +
-         for (int i = 0, m = elementsList.size(); i < m; i++)
++        if (values.isEmpty())
 +        {
-             List<ByteBuffer> oldComposite = elementsList.remove(0);
- 
-             for (int j = 0, n = values.size(); j < n; j++)
++            hasMissingElements = true;
++        }
++        else
++        {
++            for (int i = 0, m = elementsList.size(); i < m; i++)
 +            {
-                 List<ByteBuffer> newComposite = new ArrayList<>(oldComposite);
-                 elementsList.add(newComposite);
++                List<ByteBuffer> oldComposite = elementsList.remove(0);
++
++                for (int j = 0, n = values.size(); j < n; j++)
++                {
++                    List<ByteBuffer> newComposite = new ArrayList<>(oldComposite);
++                    elementsList.add(newComposite);
 +
-                 List<ByteBuffer> value = values.get(j);
++                    List<ByteBuffer> value = values.get(j);
 +
-                 if (value.contains(null))
-                     containsNull = true;
++                    if (value.isEmpty())
++                        hasMissingElements = true;
 +
-                 newComposite.addAll(value);
++                    if (value.contains(null))
++                        containsNull = true;
++
++                    newComposite.addAll(value);
++                }
 +            }
++            size += values.get(0).size();
 +        }
- 
-         remaining -= values.get(0).size();
 +        return this;
 +    }
 +
 +    /**
 +     * Returns the number of elements that can be added to the composites.
 +     *
 +     * @return the number of elements that can be added to the composites.
 +     */
 +    public int remainingCount()
 +    {
-         return remaining;
++        return ctype.size() - size;
 +    }
 +
 +    /**
 +     * Checks if some elements can still be added to the composites.
 +     *
 +     * @return <code>true</code> if it is possible to add more elements to the composites, <code>false</code> otherwise.
 +     */
 +    public boolean hasRemaining()
 +    {
-         return remaining > 0;
++        return remainingCount() > 0;
 +    }
 +
 +    /**
 +     * Checks if this builder is empty.
 +     *
 +     * @return <code>true</code> if this builder is empty, <code>false</code> otherwise.
 +     */
 +    public boolean isEmpty()
 +    {
 +        return elementsList.isEmpty();
 +    }
 +
 +    /**
 +     * Checks if the composites contains null elements.
 +     *
 +     * @return <code>true</code> if the composites contains <code>null</code> elements, <code>false</code> otherwise.
 +     */
 +    public boolean containsNull()
 +    {
 +        return containsNull;
 +    }
 +
 +    /**
++     * Checks if some empty list of values have been added
++     * @return <code>true</code> if the composites have some missing elements, <code>false</code> otherwise.
++     */
++    public boolean hasMissingElements()
++    {
++        return hasMissingElements;
++    }
++
++    /**
 +     * Builds the <code>Composites</code>.
 +     *
 +     * @return the composites
 +     */
 +    public List<Composite> build()
 +    {
 +        return buildWithEOC(EOC.NONE);
 +    }
 +
 +    /**
 +     * Builds the <code>Composites</code> with the specified EOC.
 +     *
 +     * @return the composites
 +     */
 +    public List<Composite> buildWithEOC(EOC eoc)
 +    {
 +        built = true;
 +
++        if (hasMissingElements)
++            return Collections.emptyList();
++
++        CBuilder builder = ctype.builder();
++
 +        if (elementsList.isEmpty())
 +            return singletonList(builder.build().withEOC(eoc));
 +
 +        // Use a Set to sort if needed and eliminate duplicates
 +        Set<Composite> set = newSet();
 +
 +        for (int i = 0, m = elementsList.size(); i < m; i++)
 +        {
 +            List<ByteBuffer> elements = elementsList.get(i);
 +            set.add(builder.buildWith(elements).withEOC(eoc));
 +        }
 +
 +        return new ArrayList<>(set);
 +    }
 +
 +    /**
 +     * Returns a new <code>Set</code> instance that will be used to eliminate duplicates and sort the results.
 +     *
 +     * @return a new <code>Set</code> instance.
 +     */
 +    private Set<Composite> newSet()
 +    {
-         return comparator == null ? new LinkedHashSet<Composite>() : new TreeSet<Composite>(comparator);
++        return new TreeSet<>(ctype);
 +    }
 +
 +    private void checkUpdateable()
 +    {
 +        if (!hasRemaining() || built)
 +            throw new IllegalStateException("this CompositesBuilder cannot be updated anymore");
 +    }
 +}