You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2015/08/09 22:47:54 UTC

[3/5] cassandra git commit: Allows single-column slice restrictions to be merged with multi-columns slice restrictions

Allows single-column slice restrictions to be merged with multi-columns slice restrictions

patch by Benjamin Lerer; reviewed by Sam Tunnicliffe for CASSANDRA-9606


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

Branch: refs/heads/cassandra-3.0
Commit: c8d163f73cad4cb61c12c6871e26489b52094d17
Parents: 1ae58ef
Author: blerer <be...@datastax.com>
Authored: Sun Aug 9 22:09:48 2015 +0200
Committer: blerer <be...@datastax.com>
Committed: Sun Aug 9 22:09:48 2015 +0200

----------------------------------------------------------------------
 .../cql3/statements/MultiColumnRestriction.java | 10 +++-
 .../cassandra/cql3/statements/Restriction.java  |  5 +-
 .../cql3/statements/SelectStatement.java        | 61 ++++++++++++++------
 .../statements/SingleColumnRestriction.java     | 23 +++++++-
 .../SelectMultiColumnRelationTest.java          | 44 ++++++++++----
 .../SelectSingleColumnRelationTest.java         | 28 +++++++++
 6 files changed, 138 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c8d163f7/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
index 6946c98..e2ba2a1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
@@ -25,8 +25,11 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
+import org.apache.cassandra.cql3.Term.Terminal;
+
 public interface MultiColumnRestriction extends Restriction
 {
     public static class EQ extends SingleColumnRestriction.EQ implements MultiColumnRestriction
@@ -130,8 +133,11 @@ public interface MultiColumnRestriction extends Restriction
          */
         public List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
         {
-            Tuples.Value value = (Tuples.Value)bounds[b.idx].bind(options);
-            return value.getElements();
+            Terminal terminal = bounds[b.idx].bind(options);
+            if (terminal instanceof Tuples.Value)
+                return ((Tuples.Value) terminal).getElements();
+
+            return Collections.singletonList(terminal.get(options));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c8d163f7/src/java/org/apache/cassandra/cql3/statements/Restriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Restriction.java b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
index 485fd22..bd0277c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.Term;
@@ -63,6 +62,8 @@ public interface Restriction
         /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
         public boolean hasBound(Bound b);
 
+        public Term bound(Bound b);
+
         public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException;
 
         /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
@@ -73,5 +74,7 @@ public interface Restriction
         public Operator getIndexOperator(Bound b);
 
         public void setBound(Operator type, Term t) throws InvalidRequestException;
+
+        public void setBound(Slice restriction) throws InvalidRequestException;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c8d163f7/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 26a1ee1..44b780e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -1699,13 +1699,6 @@ public class SelectStatement implements CQLStatement
                     }
                     else
                     {
-                        if (!existing.isMultiColumn())
-                        {
-                            throw new InvalidRequestException(String.format(
-                                    "Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities: %s",
-                                    def.name, relation));
-                        }
-
                         boolean existingRestrictionStartBefore =
                             (i == 0 && def.position() != 0 && stmt.columnRestrictions[def.position() - 1] == existing);
 
@@ -1714,7 +1707,7 @@ public class SelectStatement implements CQLStatement
                         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",
+                                    "Column \"%s\" cannot be restricted by two inequalities not starting with the same column: %s",
                                     def.name, relation));
                         }
 
@@ -1771,9 +1764,22 @@ public class SelectStatement implements CQLStatement
                 {
                     Term t = relation.getValue().prepare(keyspace(), defs);
                     t.collectMarkerSpecification(boundNames);
-                    Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, defs.get(0));
-                    if (restriction == null)
+
+                    Restriction.Slice existingRestriction = (Restriction.Slice) getExistingRestriction(stmt, defs.get(0));
+                    Restriction.Slice restriction;
+                    if (existingRestriction == null)
+                    {
+                        restriction = new MultiColumnRestriction.Slice(false);
+                    }
+                    else if (!existingRestriction.isMultiColumn())
+                    {
                         restriction = new MultiColumnRestriction.Slice(false);
+                        restriction.setBound(existingRestriction);
+                    }
+                    else
+                    {
+                        restriction = existingRestriction;
+                    }
                     restriction.setBound(relation.operator(), t);
 
                     for (ColumnDefinition def : defs)
@@ -1829,26 +1835,39 @@ public class SelectStatement implements CQLStatement
             switch (def.kind)
             {
                 case PARTITION_KEY:
-                    stmt.keyRestrictions[def.position()] = updateSingleColumnRestriction(def, stmt.keyRestrictions[def.position()], relation, names);
+                {
+                    Restriction existingRestriction = stmt.keyRestrictions[def.position()];
+                    Restriction previousRestriction = def.position() == 0 ? null : stmt.keyRestrictions[def.position() - 1];
+                    stmt.keyRestrictions[def.position()] = updateSingleColumnRestriction(def, existingRestriction, previousRestriction, relation, names);
                     break;
+                }
                 case CLUSTERING_COLUMN:
-                    stmt.columnRestrictions[def.position()] = updateSingleColumnRestriction(def, stmt.columnRestrictions[def.position()], relation, names);
+                {
+                    Restriction existingRestriction = stmt.columnRestrictions[def.position()];
+                    Restriction previousRestriction = def.position() == 0 ? null : stmt.columnRestrictions[def.position() - 1];
+                    stmt.columnRestrictions[def.position()] = updateSingleColumnRestriction(def, existingRestriction, previousRestriction, relation, names);
                     break;
+                }
                 case COMPACT_VALUE:
+                {
                     throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", def.name));
+                }
                 case REGULAR:
                 case STATIC:
+                {
                     // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
-                    Restriction r = updateSingleColumnRestriction(def, stmt.metadataRestrictions.get(def.name), relation, names);
+                    Restriction r = updateSingleColumnRestriction(def, stmt.metadataRestrictions.get(def.name), null, relation, names);
+
                     if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
                         // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that slide.
                         throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", def.name));
                     stmt.metadataRestrictions.put(def.name, r);
                     break;
+                }
             }
         }
 
-        Restriction updateSingleColumnRestriction(ColumnDefinition def, Restriction existingRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
+        Restriction updateSingleColumnRestriction(ColumnDefinition def, Restriction existingRestriction, Restriction previousRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
         {
             ColumnSpecification receiver = def;
             if (newRel.onToken)
@@ -1911,12 +1930,14 @@ public class SelectStatement implements CQLStatement
                 case LT:
                 case LTE:
                     {
+                        // A slice restriction can be merged with another one under some conditions:
+                        // 1) both restrictions are on a token function or non of them are
+                        // (e.g. token(partitionKey) > token(?) AND token(partitionKey) <= token(?) or clustering1 > 1 AND clustering1 <= 2).
+                        // 2) both restrictions needs to start with the same column (e.g clustering1 > 0 AND (clustering1, clustering2) <= (2, 1)).
                         if (existingRestriction == null)
                             existingRestriction = new SingleColumnRestriction.Slice(newRel.onToken);
                         else if (!existingRestriction.isSlice())
                             throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both an equality and an inequality relation", def.name));
-                        else if (existingRestriction.isMultiColumn())
-                            throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both a tuple notation inequality and a single column inequality (%s)", def.name, newRel));
                         else if (existingRestriction.isOnToken() != newRel.onToken)
                             // For partition keys, we shouldn't have slice restrictions without token(). And while this is rejected later by
                             // processPartitionKeysRestrictions, we shouldn't update the existing restriction by the new one if the old one was using token()
@@ -1925,9 +1946,14 @@ public class SelectStatement implements CQLStatement
 
                         checkBound(existingRestriction, def, newRel.operator());
 
+                        if (def.position() != 0 && previousRestriction == existingRestriction)
+                            throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by two inequalities not starting with the same column: %s",
+                                                                            def.name,
+                                                                            newRel));
+
                         Term t = newRel.getValue().prepare(keyspace(), receiver);
                         t.collectMarkerSpecification(boundNames);
-                        ((SingleColumnRestriction.Slice)existingRestriction).setBound(newRel.operator(), t);
+                        ((SingleColumnRestriction.Slice) existingRestriction).setBound(newRel.operator(), t);
                     }
                     break;
                 case CONTAINS_KEY:
@@ -1947,6 +1973,7 @@ public class SelectStatement implements CQLStatement
                     boolean isKey = newRel.operator() == Operator.CONTAINS_KEY;
                     receiver = makeCollectionReceiver(receiver, isKey);
                     Term t = newRel.getValue().prepare(keyspace(), receiver);
+
                     t.collectMarkerSpecification(boundNames);
                     ((SingleColumnRestriction.Contains)existingRestriction).add(t, isKey);
                     break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c8d163f7/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
index 34cd175..17229a6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
@@ -257,6 +257,11 @@ public abstract class SingleColumnRestriction implements Restriction
             return bounds[b.idx] != null;
         }
 
+        public Term bound(Bound b)
+        {
+            return bounds[b.idx];
+        }
+
         public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
         {
             return bounds[b.idx].bindAndGet(options);
@@ -319,10 +324,22 @@ public abstract class SingleColumnRestriction implements Restriction
                     throw new AssertionError();
             }
 
-            assert bounds[b.idx] == null;
+            setBound(b, inclusive, t);
+        }
+
+        public void setBound(Restriction.Slice slice) throws InvalidRequestException
+        {
+            for (Bound bound : Bound.values())
+                if (slice.hasBound(bound))
+                    setBound(bound, slice.isInclusive(bound), slice.bound(bound));
+        }
+
+        private void setBound(Bound bound, boolean inclusive, Term term) throws InvalidRequestException {
+
+            assert bounds[bound.idx] == null;
 
-            bounds[b.idx] = t;
-            boundInclusive[b.idx] = inclusive;
+            bounds[bound.idx] = term;
+            boundInclusive[bound.idx] = inclusive;
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c8d163f7/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
index 954f89d..98dda26 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@ -38,6 +38,8 @@ public class SelectMultiColumnRelationTest extends CQLTester
                                  "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
             assertInvalidMessage("More than one restriction was found for the start bound on b",
                                  "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
+            assertInvalidMessage("More than one restriction was found for the start bound on b",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND b > ?", 0, 1);
             assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
                                  "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
         }
@@ -86,13 +88,9 @@ public class SelectMultiColumnRelationTest extends CQLTester
                                  "SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
 
             // Mix single and tuple inequalities
-            assertInvalidMessage("Column \"b\" cannot be restricted by both a tuple notation inequality and a single column inequality (b < ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
-            assertInvalidMessage("Column \"c\" cannot be restricted by both a tuple notation inequality and a single column inequality (c < ?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: c < ?",
                                  "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-            assertInvalidMessage("Column \"b\" cannot have both tuple-notation inequalities and single-column inequalities: (b, c, d) < (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-            assertInvalidMessage("Column \"c\" cannot have both tuple-notation inequalities and single-column inequalities: (b, c, d) < (?, ?, ?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c, d) < (?, ?, ?)",
                                  "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
 
             assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
@@ -106,14 +104,16 @@ public class SelectMultiColumnRelationTest extends CQLTester
                                  "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
             assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                  "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (c) < (?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (c) < (?)",
                                  "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (b, c) > (?, ?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
                                  "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (b, c) > (?, ?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
                                  "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
+                                 "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND c < ? AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
 
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (c) < (?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (c) < (?)",
                                  "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
 
             assertInvalidMessage("PRIMARY KEY column \"d\" cannot be restricted (preceding column \"c\" is restricted by an IN tuple notation)",
@@ -162,6 +162,10 @@ public class SelectMultiColumnRelationTest extends CQLTester
                        row(0, 1, 1, 0),
                        row(0, 1, 1, 1));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and c <= ? ", 0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
                        row(0, 1, 0, 0),
                        row(0, 1, 1, 0));
@@ -231,6 +235,10 @@ public class SelectMultiColumnRelationTest extends CQLTester
                        row(0, 1, 1, 0),
                        row(0, 1, 1, 1));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and c <= ? ", 0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
                        row(0, 1, 0, 0),
                        row(0, 1, 1, 0));
@@ -302,6 +310,14 @@ public class SelectMultiColumnRelationTest extends CQLTester
             assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND (b) < (?)", 0, 0, 2),
                     row(0, 1, 0)
             );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND b < ?", 0, 0, 2),
+                       row(0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND b > ? AND (b) < (?)", 0, 0, 2),
+                       row(0, 1, 0)
+            );
         }
     }
 
@@ -445,6 +461,10 @@ public class SelectMultiColumnRelationTest extends CQLTester
                     row(0, 0, 1, 1)
             );
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 0, 1, 0, 1),
+                       row(0, 0, 1, 1)
+            );
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)", 0, 0, 1, 1, 1, 1),
                     row(0, 1, 0, 0)
             );
@@ -534,6 +554,10 @@ public class SelectMultiColumnRelationTest extends CQLTester
                     row(0, 0, 1, 1)
             );
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND b < ? ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
+                       row(0, 0, 1, 1)
+            );
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1),
                     row(0, 1, 0, 0)
             );

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c8d163f7/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
index 05bb88b..9896d0a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -55,6 +55,34 @@ public class SelectSingleColumnRelationTest extends CQLTester
     }
 
     @Test
+    public void testInvalidSliceRestrictionOnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c text)");
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "SELECT * FROM %s WHERE a >= 1 and a < 4");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s WHERE (a) >= (1) and (a) < (4)");
+    }
+
+    @Test
+    public void testInvalidMulticolumnSliceRestrictionOnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY ((a, b)))");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (a, b) < (4, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s WHERE a >= 1 and (a, b) < (4, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s WHERE b >= 1 and (a, b) < (4, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (b) < (4)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: b",
+                             "SELECT * FROM %s WHERE (b) < (4) and (a, b) >= (1, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and a = 1");
+    }
+
+    @Test
     public void testLargeClusteringINValues() throws Throwable
     {
         createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");