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:25:46 UTC

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

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 fb67c41ad -> 90a012a1f


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


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

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

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


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

Branch: refs/heads/cassandra-2.0
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"})