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:17:31 UTC

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

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 0b99d33ee -> 6d0cf7dbd


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/de84a5c7
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/de84a5c7
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/de84a5c7

Branch: refs/heads/cassandra-2.2
Commit: de84a5c770ac1a429152dd79f0895b27aa544368
Parents: c9587cd
Author: blerer <be...@datastax.com>
Authored: Sun Aug 9 21:48:04 2015 +0200
Committer: blerer <be...@datastax.com>
Committed: Sun Aug 9 21:48:04 2015 +0200

----------------------------------------------------------------------
 .../cql3/statements/MultiColumnRestriction.java | 10 +++-
 .../cassandra/cql3/statements/Restriction.java  |  4 ++
 .../cql3/statements/SelectStatement.java        | 54 ++++++++++++++------
 .../statements/SingleColumnRestriction.java     | 25 +++++++--
 .../cassandra/cql3/MultiColumnRelationTest.java | 49 +++++++++++++++++-
 .../cql3/SingleColumnRelationTest.java          | 53 +++++++++++++++++--
 6 files changed, 165 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/de84a5c7/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 f643684..ed1e7f1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
@@ -24,8 +24,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
@@ -128,8 +131,11 @@ public interface MultiColumnRestriction extends Restriction
          */
         public List<ByteBuffer> componentBounds(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
         {
-            Tuples.Value value = (Tuples.Value)bounds[b.idx].bind(variables);
-            return value.getElements();
+            Terminal terminal = bounds[b.idx].bind(variables);
+            if (terminal instanceof Tuples.Value)
+                return ((Tuples.Value) terminal).getElements();
+
+            return Collections.singletonList(terminal.get());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/de84a5c7/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 3d33bde..f582c84 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@ -54,6 +54,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, List<ByteBuffer> variables) throws InvalidRequestException;
 
         /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
@@ -64,5 +66,7 @@ public interface Restriction
         public IndexOperator getIndexOperator(Bound b);
 
         public void setBound(Relation.Type type, Term t) throws InvalidRequestException;
+
+        public void setBound(Slice restriction) throws InvalidRequestException;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/de84a5c7/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 aaf9579..a9eae7a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -46,7 +46,6 @@ import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.pager.*;
 import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.thrift.ColumnDef;
 import org.apache.cassandra.thrift.IndexExpression;
 import org.apache.cassandra.thrift.IndexOperator;
 import org.apache.cassandra.thrift.ThriftValidation;
@@ -1718,13 +1717,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     }
                     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);
 
@@ -1733,7 +1725,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                         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 inequalities not starting with the same column: %s",
                                     name, relation));
                         }
 
@@ -1793,9 +1785,21 @@ 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.Slice existingRestriction = (Restriction.Slice) getExistingRestriction(stmt, names.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 (CFDefinition.Name name : names)
@@ -1847,17 +1851,25 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             switch (name.kind)
             {
                 case KEY_ALIAS:
-                    stmt.keyRestrictions[name.position] = updateSingleColumnRestriction(name, stmt.keyRestrictions[name.position], relation, names);
+                {
+                    Restriction existingRestriction = stmt.keyRestrictions[name.position];
+                    Restriction previousRestriction = name.position == 0 ? null : stmt.keyRestrictions[name.position - 1];
+                    stmt.keyRestrictions[name.position] = updateSingleColumnRestriction(name, existingRestriction, previousRestriction, relation, names);
                     break;
+                }
                 case COLUMN_ALIAS:
-                    stmt.columnRestrictions[name.position] = updateSingleColumnRestriction(name, stmt.columnRestrictions[name.position], relation, names);
+                {
+                    Restriction existingRestriction = stmt.columnRestrictions[name.position];
+                    Restriction previousRestriction = name.position == 0 ? null : stmt.columnRestrictions[name.position - 1];
+                    stmt.columnRestrictions[name.position] = updateSingleColumnRestriction(name, existingRestriction, previousRestriction, relation, names);
                     break;
+                }
                 case VALUE_ALIAS:
                     throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", name.name));
                 case COLUMN_METADATA:
                 case STATIC:
                     // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
-                    Restriction r = updateSingleColumnRestriction(name, stmt.metadataRestrictions.get(name), relation, names);
+                    Restriction r = updateSingleColumnRestriction(name, stmt.metadataRestrictions.get(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", name));
@@ -1866,7 +1878,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             }
         }
 
-        Restriction updateSingleColumnRestriction(CFDefinition.Name name, Restriction existingRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
+        Restriction updateSingleColumnRestriction(CFDefinition.Name name, Restriction existingRestriction, Restriction previousRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
         {
             ColumnSpecification receiver = name;
             if (newRel.onToken)
@@ -1927,6 +1939,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 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())
@@ -1936,8 +1952,12 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                         // processPartitionKeysRestrictions, we shouldn't update the existing restriction by the new one if the old one was using token()
                         // and the new one isn't since that would bypass that later test.
                         throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
-                    else if (existingRestriction.isMultiColumn())
-                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both a tuple notation inequality and a single column inequality (%s)", name, newRel));
+
+                    if (name.position != 0 && previousRestriction == existingRestriction)
+                        throw new InvalidRequestException(String.format(
+                                "Column \"%s\" cannot be restricted by two inequalities not starting with the same column: %s",
+                                name, newRel));
+
                     Term t = newRel.getValue().prepare(receiver);
                     t.collectMarkerSpecification(boundNames);
                     ((SingleColumnRestriction.Slice)existingRestriction).setBound(newRel.operator(), t);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/de84a5c7/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 2e63272..e326597 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
@@ -218,6 +218,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, List<ByteBuffer> variables) throws InvalidRequestException
         {
             return bounds[b.idx].bindAndGet(variables);
@@ -279,12 +284,24 @@ public abstract class SingleColumnRestriction implements Restriction
                     throw new AssertionError();
             }
 
-            if (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 {
+
+            if (bounds[bound.idx] != null)
                 throw new InvalidRequestException(String.format(
-                        "More than one restriction was found for the %s bound", b.name().toLowerCase()));
+                        "More than one restriction was found for the %s bound", bound.name().toLowerCase()));
 
-            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/de84a5c7/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 65ff3e7..ac3d882 100644
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
@@ -296,9 +296,7 @@ public class MultiColumnRelationTest
         for (String tableSuffix : new String[]{"", "_compact"})
         {
             String[] queries = new String[]{
-                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c, d) > (0, 1, 0) AND b < 1",
                 "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c, d) > (0, 1, 0) AND c < 1",
-                "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND b > 1 AND (b, c, d) < (1, 1, 0)",
                 "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND c > 1 AND (b, c, d) < (1, 1, 0)",
                 "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE (a, b, c, d) IN ((0, 1, 2, 3))",
                 "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE (c, d) IN ((0, 1))",
@@ -380,6 +378,12 @@ public class MultiColumnRelationTest
             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);
@@ -477,6 +481,12 @@ public class MultiColumnRelationTest
             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);
@@ -516,6 +526,14 @@ public class MultiColumnRelationTest
             results = execute("SELECT * FROM %s.single_clustering" + tableSuffix + " WHERE a=0 AND (b) > (0) AND (b) < (2)");
             assertEquals(1, results.size());
             checkRow(0, results, 0, 1, 0);
+
+            results = execute("SELECT * FROM %s.single_clustering" + tableSuffix + " WHERE a=0 AND b > 0 AND (b) < (2)");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0);
+
+            results = execute("SELECT * FROM %s.single_clustering" + tableSuffix + " WHERE a=0 AND (b) > (0) AND b < 2");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0);
         }
     }
 
@@ -606,6 +624,10 @@ public class MultiColumnRelationTest
             assertEquals(1, results.size());
             checkRow(0, results, 0, 0, 1, 1);
 
+            results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > (0, 1, 0) AND b < 1");
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 0, 1, 1);
+
             results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c) < (1, 1)");
             assertEquals(1, results.size());
             checkRow(0, results, 0, 1, 0, 0);
@@ -973,6 +995,12 @@ public class MultiColumnRelationTest
     }
 
     @Test(expected=InvalidRequestException.class)
+    public void testPrepareMixMultipleInequalitiesOnSameBoundWithSingleColumnRestriction() throws Throwable
+    {
+        prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?) AND b > ?");
+    }
+
+    @Test(expected=InvalidRequestException.class)
     public void testPrepareClusteringColumnsOutOfOrderInInequality() throws Throwable
     {
         prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) > (?, ?, ?)");
@@ -1051,6 +1079,14 @@ public class MultiColumnRelationTest
             results = executePrepared(prepare("SELECT * FROM %s.single_clustering" + tableSuffix + " WHERE a=0 AND (b) > (?) AND (b) < (?)"), makeIntOptions(0, 2));
             assertEquals(1, results.size());
             checkRow(0, results, 0, 1, 0);
+
+            results = executePrepared(prepare("SELECT * FROM %s.single_clustering" + tableSuffix + " WHERE a=0 AND (b) > (?) AND b < ?"), makeIntOptions(0, 2));
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0);
+
+            results = executePrepared(prepare("SELECT * FROM %s.single_clustering" + tableSuffix + " WHERE a=0 AND b > ? AND (b) < (?)"), makeIntOptions(0, 2));
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 1, 0);
         }
     }
 
@@ -1129,6 +1165,10 @@ public class MultiColumnRelationTest
             assertEquals(1, results.size());
             checkRow(0, results, 0, 0, 1, 1);
 
+            results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > (?, ?, ?) AND b < ?"), makeIntOptions(0, 1, 0, 1));
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 0, 1, 1);
+
             results = executePrepared(prepare
                             ("SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)"),
                     makeIntOptions(0, 1, 1, 1, 1));
@@ -1195,6 +1235,11 @@ public class MultiColumnRelationTest
             assertEquals(1, results.size());
             checkRow(0, results, 0, 0, 1, 1);
 
+            results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering" + tableSuffix
+                    + " WHERE a=0 AND (b, c, d) > ? AND b < ?"), options(tuple(0, 1, 0), ByteBufferUtil.bytes(1)));
+            assertEquals(1, results.size());
+            checkRow(0, results, 0, 0, 1, 1);
+
             results = executePrepared(prepare
                             ("SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > ? AND (b, c) < ?"),
                     options(tuple(0, 1, 1), tuple(1, 1)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/de84a5c7/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
index c8c67aa..8ce4a36 100644
--- a/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
@@ -21,6 +21,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -45,7 +46,8 @@ public class SingleColumnRelationTest
     {
         SchemaLoader.loadSchema();
         executeSchemaChange("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
-
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_partition (a int PRIMARY KEY, b int, c text)");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.compound_partition (a int, b int, c text, PRIMARY KEY ((a, b)))");
         executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.partition_with_indices (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
         executeSchemaChange("CREATE INDEX ON %s.partition_with_indices (c)");
         executeSchemaChange("CREATE INDEX ON %s.partition_with_indices (f)");
@@ -133,16 +135,44 @@ public class SingleColumnRelationTest
         checkRow(0, results, 0, 0, 1, 1, 1, 5);
     }
 
-    @Test(expected=InvalidRequestException.class)
+    @Test
+    public void testSliceRestrictionOnPartitionKey() throws Throwable
+    {
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "SELECT * FROM %s.single_partition WHERE a >= 1 and a < 4");
+    }
+
+    @Test
+    public void testMulticolumnSliceRestrictionOnPartitionKey() throws Throwable
+    {
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s.single_partition WHERE (a) >= (1) and (a) < (4)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s.compound_partition 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.compound_partition WHERE a >= 1 and (a, b) < (4, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s.compound_partition WHERE b >= 1 and (a, b) < (4, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s.compound_partition WHERE (a, b) >= (1, 1) and (b) < (4)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: b",
+                             "SELECT * FROM %s.compound_partition WHERE (b) < (4) and (a, b) >= (1, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                             "SELECT * FROM %s.compound_partition WHERE (a, b) >= (1, 1) and a = 1");
+    }
+
+    @Test
     public void testMissingPartitionComponentAndFileringOnTheSecondClusteringColumnWithoutAllowFiltering() throws Throwable
     {
-        execute("SELECT * FROM %s.partition_with_indices WHERE d >= 1 AND f = 5");
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s.partition_with_indices WHERE d >= 1 AND f = 5");
     }
 
-    @Test(expected=InvalidRequestException.class)
+    @Test
     public void testMissingPartitionComponentWithSliceRestrictionOnIndexedColumn() throws Throwable
     {
-        execute("SELECT * FROM %s.partition_with_indices WHERE a = 0 AND c >= 1 ALLOW FILTERING");
+        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
+                             "SELECT * FROM %s.partition_with_indices WHERE a = 0 AND c >= 1 ALLOW FILTERING");
     }
 
     private static void checkRow(int rowIndex, UntypedResultSet results, Integer... expectedValues)
@@ -158,4 +188,17 @@ public class SingleColumnRelationTest
                          (long) expected, actual);
         }
     }
+
+    private static void assertInvalidMessage(String expectedMsg, String query) throws Throwable
+    {
+        try
+        {
+            execute(query);
+            Assert.fail("The statement should trigger an InvalidRequestException but did not");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertEquals("The error message is not the expected one.",expectedMsg, e.getMessage());
+        }
+    }
 }


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

Posted by bl...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-2.2
Commit: 6d0cf7dbd2221c13434958e9e0d5e418f58fe23f
Parents: 0b99d33 c8d163f
Author: blerer <be...@datastax.com>
Authored: Sun Aug 9 22:11:14 2015 +0200
Committer: blerer <be...@datastax.com>
Committed: Sun Aug 9 22:12:16 2015 +0200

----------------------------------------------------------------------
 .../restrictions/MultiColumnRestriction.java    | 37 +++++++----
 .../restrictions/SingleColumnRestriction.java   | 68 ++++++++++++++++++--
 .../SelectMultiColumnRelationTest.java          | 45 +++++++++----
 .../SelectSingleColumnRelationTest.java         | 28 ++++++++
 4 files changed, 149 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d0cf7db/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
index c4bce4c,0000000..84a3952
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
@@@ -1,448 -1,0 +1,461 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
++import org.apache.cassandra.cql3.Term.Terminal;
++
++import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
 +import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
- 
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
 +public abstract class MultiColumnRestriction extends AbstractRestriction
 +{
 +    /**
 +     * The columns to which the restriction apply.
 +     */
 +    protected final List<ColumnDefinition> columnDefs;
 +
 +    public MultiColumnRestriction(List<ColumnDefinition> columnDefs)
 +    {
 +        this.columnDefs = columnDefs;
 +    }
 +
 +    @Override
 +    public boolean isMultiColumn()
 +    {
 +        return true;
 +    }
 +
 +    @Override
 +    public ColumnDefinition getFirstColumn()
 +    {
 +        return columnDefs.get(0);
 +    }
 +
 +    @Override
 +    public ColumnDefinition getLastColumn()
 +    {
 +        return columnDefs.get(columnDefs.size() - 1);
 +    }
 +
 +    @Override
 +    public Collection<ColumnDefinition> getColumnDefs()
 +    {
 +        return columnDefs;
 +    }
 +
 +    @Override
 +    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
 +    {
-             checkTrue(otherRestriction.isMultiColumn(),
-                       "Mixing single column relations and multi column relations on clustering columns is not allowed");
-             return doMergeWith(otherRestriction);
++        // We want to allow query like: (b,c) > (?, ?) AND b < ?
++        if (!otherRestriction.isMultiColumn()
++                && ((SingleColumnRestriction) otherRestriction).canBeConvertedToMultiColumnRestriction())
++        {
++            return doMergeWith(((SingleColumnRestriction) otherRestriction).toMultiColumnRestriction());
++        }
++
++        return doMergeWith(otherRestriction);
 +    }
 +
 +    protected abstract Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException;
 +
 +    /**
 +     * Returns the names of the columns that are specified within this <code>Restrictions</code> and the other one
 +     * as a comma separated <code>String</code>.
 +     *
 +     * @param otherRestriction the other restrictions
 +     * @return the names of the columns that are specified within this <code>Restrictions</code> and the other one
 +     * as a comma separated <code>String</code>.
 +     */
 +    protected final String getColumnsInCommons(Restriction otherRestriction)
 +    {
 +        Set<ColumnDefinition> commons = new HashSet<>(getColumnDefs());
 +        commons.retainAll(otherRestriction.getColumnDefs());
 +        StringBuilder builder = new StringBuilder();
 +        for (ColumnDefinition columnDefinition : commons)
 +        {
 +            if (builder.length() != 0)
 +                builder.append(" ,");
 +            builder.append(columnDefinition.name);
 +        }
 +        return builder.toString();
 +    }
 +
 +    @Override
 +    public final boolean hasSupportingIndex(SecondaryIndexManager indexManager)
 +    {
 +        for (ColumnDefinition columnDef : columnDefs)
 +        {
 +            SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
 +            if (index != null && isSupportedBy(index))
 +                return true;
 +        }
 +        return false;
 +    }
 +
 +    /**
 +     * Check if this type of restriction is supported for by the specified index.
 +     * @param index the Secondary index
 +     *
 +     * @return <code>true</code> this type of restriction is supported by the specified index,
 +     * <code>false</code> otherwise.
 +     */
 +    protected abstract boolean isSupportedBy(SecondaryIndex index);
 +
 +    public static class EQ  extends MultiColumnRestriction
 +    {
 +        protected final Term value;
 +
 +        public EQ(List<ColumnDefinition> columnDefs, Term value)
 +        {
 +            super(columnDefs);
 +            this.value = value;
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return value.getFunctions();
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("EQ(%s)", value);
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
 +                                 getColumnsInCommons(otherRestriction));
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.EQ);
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            Tuples.Value t = ((Tuples.Value) value.bind(options));
 +            List<ByteBuffer> values = t.getElements();
 +            for (int i = 0, m = values.size(); i < m; i++)
 +            {
 +                builder.addElementToAll(values.get(i));
 +                checkFalse(builder.containsNull(), "Invalid null value for column %s", columnDefs.get(i).name);
 +            }
 +            return builder;
 +        }
 +
 +        @Override
 +        public final void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                               SecondaryIndexManager indexManager,
 +                                               QueryOptions options) throws InvalidRequestException
 +        {
 +            Tuples.Value t = ((Tuples.Value) value.bind(options));
 +            List<ByteBuffer> values = t.getElements();
 +
 +            for (int i = 0, m = columnDefs.size(); i < m; i++)
 +            {
 +                ColumnDefinition columnDef = columnDefs.get(i);
 +                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
 +                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
 +            }
 +        }
 +    }
 +
 +    public abstract static class IN extends MultiColumnRestriction
 +    {
 +        /**
 +         * {@inheritDoc}
 +         */
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            List<List<ByteBuffer>> splitInValues = splitValues(options);
 +            builder.addAllElementsToAll(splitInValues);
 +
 +            if (builder.containsNull())
 +                throw invalidRequest("Invalid null value in condition for columns: %s", ColumnDefinition.toIdentifiers(columnDefs));
 +            return builder;
 +        }
 +
 +        public IN(List<ColumnDefinition> columnDefs)
 +        {
 +            super(columnDefs);
 +        }
 +
 +        @Override
 +        public boolean isIN()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN",
 +                                 getColumnsInCommons(otherRestriction));
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.IN);
 +        }
 +
 +        @Override
 +        public final void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                               SecondaryIndexManager indexManager,
 +                                               QueryOptions options) throws InvalidRequestException
 +        {
 +            List<List<ByteBuffer>> splitInValues = splitValues(options);
 +            checkTrue(splitInValues.size() == 1, "IN restrictions are not supported on indexed columns");
 +            List<ByteBuffer> values = splitInValues.get(0);
 +
 +            for (int i = 0, m = columnDefs.size(); i < m; i++)
 +            {
 +                ColumnDefinition columnDef = columnDefs.get(i);
 +                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
 +                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
 +            }
 +        }
 +
 +        protected abstract List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
 +    }
 +
 +    /**
 +     * An IN restriction that has a set of terms for in values.
 +     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
 +     */
 +    public static class InWithValues extends MultiColumnRestriction.IN
 +    {
 +        protected final List<Term> values;
 +
 +        public InWithValues(List<ColumnDefinition> columnDefs, List<Term> values)
 +        {
 +            super(columnDefs);
 +            this.values = values;
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Terms.getFunctions(values);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("IN(%s)", values);
 +        }
 +
 +        @Override
 +        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
 +            for (Term value : values)
 +            {
 +                Term.MultiItemTerminal term = (Term.MultiItemTerminal) value.bind(options);
 +                buffers.add(term.getElements());
 +            }
 +            return buffers;
 +        }
 +    }
 +
 +    /**
 +     * An IN restriction that uses a single marker for a set of IN values that are tuples.
 +     * For example: "SELECT ... WHERE (a, b, c) IN ?"
 +     */
 +    public static class InWithMarker extends MultiColumnRestriction.IN
 +    {
 +        protected final AbstractMarker marker;
 +
 +        public InWithMarker(List<ColumnDefinition> columnDefs, AbstractMarker marker)
 +        {
 +            super(columnDefs);
 +            this.marker = marker;
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Collections.emptySet();
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "IN ?";
 +        }
 +
 +        @Override
 +        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            Tuples.InMarker inMarker = (Tuples.InMarker) marker;
 +            Tuples.InValue inValue = inMarker.bind(options);
 +            checkNotNull(inValue, "Invalid null value for IN restriction");
 +            return inValue.getSplitValues();
 +        }
 +    }
 +
 +    public static class Slice extends MultiColumnRestriction
 +    {
 +        private final TermSlice slice;
 +
 +        public Slice(List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
 +        {
 +            this(columnDefs, TermSlice.newInstance(bound, inclusive, term));
 +        }
 +
-         private Slice(List<ColumnDefinition> columnDefs, TermSlice slice)
++        Slice(List<ColumnDefinition> columnDefs, TermSlice slice)
 +        {
 +            super(columnDefs);
 +            this.slice = slice;
 +        }
 +
 +        @Override
 +        public boolean isSlice()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
 +            List<ByteBuffer> vals = componentBounds(bound, options);
 +
 +            for (int i = 0, m = vals.size(); i < m; i++)
 +            {
 +                ByteBuffer v = checkNotNull(vals.get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
 +                builder.addElementToAll(v);
 +            }
 +            return builder;
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return slice.isSupportedBy(index);
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            return slice.hasBound(b);
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return slice.getFunctions();
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            return slice.isInclusive(b);
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isSlice(),
 +                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
 +                      getColumnsInCommons(otherRestriction));
 +
-             Slice otherSlice = (Slice) otherRestriction;
- 
 +            if (!getFirstColumn().equals(otherRestriction.getFirstColumn()))
 +            {
 +                ColumnDefinition column = getFirstColumn().position() > otherRestriction.getFirstColumn().position()
 +                        ? getFirstColumn() : otherRestriction.getFirstColumn();
 +
-                 throw invalidRequest("Column \"%s\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
++                throw invalidRequest("Column \"%s\" cannot be restricted by two inequalities not starting with the same column",
 +                                     column.name);
 +            }
 +
-             checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
++            checkFalse(hasBound(Bound.START) && otherRestriction.hasBound(Bound.START),
 +                       "More than one restriction was found for the start bound on %s",
 +                       getColumnsInCommons(otherRestriction));
-             checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
++            checkFalse(hasBound(Bound.END) && otherRestriction.hasBound(Bound.END),
 +                       "More than one restriction was found for the end bound on %s",
 +                       getColumnsInCommons(otherRestriction));
 +
++            Slice otherSlice = (Slice) otherRestriction;
 +            List<ColumnDefinition> newColumnDefs = columnDefs.size() >= otherSlice.columnDefs.size() ?  columnDefs : otherSlice.columnDefs;
++
 +            return new Slice(newColumnDefs, slice.merge(otherSlice.slice));
 +        }
 +
 +        @Override
 +        public final void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                               SecondaryIndexManager indexManager,
 +                                               QueryOptions options) throws InvalidRequestException
 +        {
 +            throw invalidRequest("Slice restrictions are not supported on indexed columns");
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "SLICE" + slice;
 +        }
 +
 +        /**
 +         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
 +         * ByteBuffer to represent the entire bound.
 +         * @param b the bound type
 +         * @param options the query options
 +         * @return one ByteBuffer per-component in the bound
 +         * @throws InvalidRequestException if the components cannot be retrieved
 +         */
 +        private List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
 +        {
-             Tuples.Value value = (Tuples.Value) slice.bound(b).bind(options);
-             return value.getElements();
++            Terminal terminal = slice.bound(b).bind(options);
++
++            if (terminal instanceof Tuples.Value)
++            {
++                return ((Tuples.Value) terminal).getElements();
++            }
++
++            return Collections.singletonList(terminal.get(options.getProtocolVersion()));
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d0cf7db/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
index d32f585,0000000..afa0419
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
@@@ -1,565 -1,0 +1,621 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.Term.Terminal;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
 +import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.db.marshal.CompositeType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
- 
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkBindValueSet;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
 +public abstract class SingleColumnRestriction extends AbstractRestriction
 +{
 +    /**
 +     * The definition of the column to which apply the restriction.
 +     */
 +    protected final ColumnDefinition columnDef;
 +
 +    public SingleColumnRestriction(ColumnDefinition columnDef)
 +    {
 +        this.columnDef = columnDef;
 +    }
 +
 +    @Override
 +    public Collection<ColumnDefinition> getColumnDefs()
 +    {
 +        return Collections.singletonList(columnDef);
 +    }
 +
 +    @Override
 +    public ColumnDefinition getFirstColumn()
 +    {
 +        return columnDef;
 +    }
 +
 +    @Override
 +    public ColumnDefinition getLastColumn()
 +    {
 +        return columnDef;
 +    }
 +
 +    @Override
 +    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
 +    {
 +        SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
 +        return index != null && isSupportedBy(index);
 +    }
 +
 +    @Override
 +    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
 +    {
-             checkFalse(otherRestriction.isMultiColumn(),
-                        "Mixing single column relations and multi column relations on clustering columns is not allowed");
++        // We want to allow query like: b > ? AND (b,c) < (?, ?)
++        if (otherRestriction.isMultiColumn() && canBeConvertedToMultiColumnRestriction())
++        {
++            return toMultiColumnRestriction().mergeWith(otherRestriction);
++        }
 +
-             return doMergeWith(otherRestriction);
++        return doMergeWith(otherRestriction);
 +    }
 +
 +    protected abstract Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException;
 +
 +    /**
++     * Converts this <code>SingleColumnRestriction</code> into a {@link MultiColumnRestriction}
++     *
++     * @return the <code>MultiColumnRestriction</code> corresponding to this
++     */
++    abstract MultiColumnRestriction toMultiColumnRestriction();
++
++    /**
++     * Checks if this <code>Restriction</code> can be converted into a {@link MultiColumnRestriction}
++     *
++     * @return <code>true</code> if this <code>Restriction</code> can be converted into a
++     * {@link MultiColumnRestriction}, <code>false</code> otherwise.
++     */
++    boolean canBeConvertedToMultiColumnRestriction()
++    {
++        return true;
++    }
++
++    /**
 +     * Check if this type of restriction is supported by the specified index.
 +     *
 +     * @param index the Secondary index
 +     * @return <code>true</code> this type of restriction is supported by the specified index,
 +     * <code>false</code> otherwise.
 +     */
 +    protected abstract boolean isSupportedBy(SecondaryIndex index);
 +
 +    public static final class EQ extends SingleColumnRestriction
 +    {
 +        private final Term value;
 +
 +        public EQ(ColumnDefinition columnDef, Term value)
 +        {
 +            super(columnDef);
 +            this.value = value;
 +        }
 +
 +        @Override
-         public Iterable getFunctions()
++        public Iterable<Function> getFunctions()
 +        {
 +            return value.getFunctions();
 +        }
 +
 +        @Override
 +        public boolean isEQ()
 +        {
 +            return true;
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            return new MultiColumnRestriction.EQ(Collections.singletonList(columnDef), value);
++        }
++
++        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options) throws InvalidRequestException
 +        {
 +            ByteBuffer buffer = validateIndexedValue(columnDef, value.bindAndGet(options));
 +            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, buffer));
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            builder.addElementToAll(value.bindAndGet(options));
 +            checkFalse(builder.containsNull(), "Invalid null value in condition for column %s", columnDef.name);
 +            checkFalse(builder.containsUnset(), "Invalid unset value for column %s", columnDef.name);
 +            return builder;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("EQ(%s)", value);
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal", columnDef.name);
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.EQ);
 +        }
 +    }
 +
 +    public static abstract class IN extends SingleColumnRestriction
 +    {
 +        public IN(ColumnDefinition columnDef)
 +        {
 +            super(columnDef);
 +        }
 +
 +        @Override
 +        public final boolean isIN()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public final Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN", columnDef.name);
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            builder.addEachElementToAll(getValues(options));
 +            checkFalse(builder.containsNull(), "Invalid null value in condition for column %s", columnDef.name);
 +            checkFalse(builder.containsUnset(), "Invalid unset value for column %s", columnDef.name);
 +            return builder;
 +        }
 +
 +        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> values = getValues(options);
 +            checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns");
 +
 +            ByteBuffer value = validateIndexedValue(columnDef, values.get(0));
 +            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, value));
 +        }
 +
 +        @Override
 +        protected final boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.IN);
 +        }
 +
 +        protected abstract List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException;
 +    }
 +
 +    public static class InWithValues extends IN
 +    {
 +        protected final List<Term> values;
 +
 +        public InWithValues(ColumnDefinition columnDef, List<Term> values)
 +        {
 +            super(columnDef);
 +            this.values = values;
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            return new MultiColumnRestriction.InWithValues(Collections.singletonList(columnDef), values);
++        }
++
++        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Terms.getFunctions(values);
 +        }
 +
 +        @Override
 +        protected List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> buffers = new ArrayList<>(values.size());
 +            for (Term value : values)
 +                buffers.add(value.bindAndGet(options));
 +            return buffers;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("IN(%s)", values);
 +        }
 +    }
 +
 +    public static class InWithMarker extends IN
 +    {
 +        protected final AbstractMarker marker;
 +
 +        public InWithMarker(ColumnDefinition columnDef, AbstractMarker marker)
 +        {
 +            super(columnDef);
 +            this.marker = marker;
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Collections.emptySet();
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            return new MultiColumnRestriction.InWithMarker(Collections.singletonList(columnDef), marker);
++        }
++
++        @Override
 +        protected List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            Terminal term = marker.bind(options);
 +            checkNotNull(term, "Invalid null value for column %s", columnDef.name);
 +            checkFalse(term == Constants.UNSET_VALUE, "Invalid unset value for column %s", columnDef.name);
 +            Term.MultiItemTerminal lval = (Term.MultiItemTerminal) term;
 +            return lval.getElements();
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "IN ?";
 +        }
 +    }
 +
-     public static class Slice extends SingleColumnRestriction
++    public static final class Slice extends SingleColumnRestriction
 +    {
 +        private final TermSlice slice;
 +
 +        public Slice(ColumnDefinition columnDef, Bound bound, boolean inclusive, Term term)
 +        {
 +            super(columnDef);
 +            slice = TermSlice.newInstance(bound, inclusive, term);
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return slice.getFunctions();
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            return new MultiColumnRestriction.Slice(Collections.singletonList(columnDef), slice);
++        }
++
++        @Override
 +        public boolean isSlice()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            return slice.hasBound(b);
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
 +            ByteBuffer value = slice.bound(bound).bindAndGet(options);
 +            checkBindValueSet(value, "Invalid unset value for column %s", columnDef.name);
 +            return builder.addElementToAll(value);
 +
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            return slice.isInclusive(b);
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isSlice(),
 +                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
 +                      columnDef.name);
 +
 +            SingleColumnRestriction.Slice otherSlice = (SingleColumnRestriction.Slice) otherRestriction;
 +
 +            checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
 +                       "More than one restriction was found for the start bound on %s", columnDef.name);
 +
 +            checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
 +                       "More than one restriction was found for the end bound on %s", columnDef.name);
 +
 +            return new Slice(columnDef,  slice.merge(otherSlice.slice));
 +        }
 +
 +        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options) throws InvalidRequestException
 +        {
 +            for (Bound b : Bound.values())
 +            {
 +                if (hasBound(b))
 +                {
 +                    ByteBuffer value = validateIndexedValue(columnDef, slice.bound(b).bindAndGet(options));
 +                    Operator op = slice.getIndexOperator(b);
 +                    // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
 +                    // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
 +                    // use the underlying comparator as is.
 +                    op = columnDef.isReversedType() ? op.reverse() : op;
 +                    expressions.add(new IndexExpression(columnDef.name.bytes, op, value));
 +                }
 +            }
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return slice.isSupportedBy(index);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("SLICE%s", slice);
 +        }
 +
 +        private Slice(ColumnDefinition columnDef, TermSlice slice)
 +        {
 +            super(columnDef);
 +            this.slice = slice;
 +        }
 +    }
 +
 +    // This holds CONTAINS, CONTAINS_KEY, and map[key] = value restrictions because we might want to have any combination of them.
 +    public static final class Contains extends SingleColumnRestriction
 +    {
 +        private List<Term> values = new ArrayList<>(); // for CONTAINS
 +        private List<Term> keys = new ArrayList<>(); // for CONTAINS_KEY
 +        private List<Term> entryKeys = new ArrayList<>(); // for map[key] = value
 +        private List<Term> entryValues = new ArrayList<>(); // for map[key] = value
 +
 +        public Contains(ColumnDefinition columnDef, Term t, boolean isKey)
 +        {
 +            super(columnDef);
 +            if (isKey)
 +                keys.add(t);
 +            else
 +                values.add(t);
 +        }
 +
 +        public Contains(ColumnDefinition columnDef, Term mapKey, Term mapValue)
 +        {
 +            super(columnDef);
 +            entryKeys.add(mapKey);
 +            entryValues.add(mapValue);
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            throw new UnsupportedOperationException();
++        }
++
++        @Override
++        boolean canBeConvertedToMultiColumnRestriction()
++        {
++            return false;
++        }
++
++        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean isContains()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isContains(),
 +                      "Collection column %s can only be restricted by CONTAINS, CONTAINS KEY, or map-entry equality",
 +                      columnDef.name);
 +
 +            SingleColumnRestriction.Contains newContains = new Contains(columnDef);
 +
 +            copyKeysAndValues(this, newContains);
 +            copyKeysAndValues((Contains) otherRestriction, newContains);
 +
 +            return newContains;
 +        }
 +
 +        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options)
 +                                         throws InvalidRequestException
 +        {
 +            addExpressionsFor(expressions, bindAndGet(values, options), Operator.CONTAINS);
 +            addExpressionsFor(expressions, bindAndGet(keys, options), Operator.CONTAINS_KEY);
 +            addExpressionsFor(expressions, entries(options), Operator.EQ);
 +        }
 +
 +        private void addExpressionsFor(List<IndexExpression> target, List<ByteBuffer> values,
 +                                       Operator op) throws InvalidRequestException
 +        {
 +            for (ByteBuffer value : values)
 +            {
 +                validateIndexedValue(columnDef, value);
 +                target.add(new IndexExpression(columnDef.name.bytes, op, value));
 +            }
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            boolean supported = false;
 +
 +            if (numberOfValues() > 0)
 +                supported |= index.supportsOperator(Operator.CONTAINS);
 +
 +            if (numberOfKeys() > 0)
 +                supported |= index.supportsOperator(Operator.CONTAINS_KEY);
 +
 +            if (numberOfEntries() > 0)
 +                supported |= index.supportsOperator(Operator.EQ);
 +
 +            return supported;
 +        }
 +
 +        public int numberOfValues()
 +        {
 +            return values.size();
 +        }
 +
 +        public int numberOfKeys()
 +        {
 +            return keys.size();
 +        }
 +
 +        public int numberOfEntries()
 +        {
 +            return entryKeys.size();
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Iterables.concat(Terms.getFunctions(values),
 +                                    Terms.getFunctions(keys),
 +                                    Terms.getFunctions(entryKeys),
 +                                    Terms.getFunctions(entryValues));
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("CONTAINS(values=%s, keys=%s, entryKeys=%s, entryValues=%s)", values, keys, entryKeys, entryValues);
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        private List<ByteBuffer> entries(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> entryBuffers = new ArrayList<>(entryKeys.size());
 +            List<ByteBuffer> keyBuffers = bindAndGet(entryKeys, options);
 +            List<ByteBuffer> valueBuffers = bindAndGet(entryValues, options);
 +            for (int i = 0; i < entryKeys.size(); i++)
 +            {
 +                if (valueBuffers.get(i) == null)
 +                    throw new InvalidRequestException("Unsupported null value for map-entry equality");
 +                entryBuffers.add(CompositeType.build(keyBuffers.get(i), valueBuffers.get(i)));
 +            }
 +            return entryBuffers;
 +        }
 +
 +        /**
 +         * Binds the query options to the specified terms and returns the resulting values.
 +         *
 +         * @param terms the terms
 +         * @param options the query options
 +         * @return the value resulting from binding the query options to the specified terms
 +         * @throws InvalidRequestException if a problem occurs while binding the query options
 +         */
 +        private static List<ByteBuffer> bindAndGet(List<Term> terms, QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> buffers = new ArrayList<>(terms.size());
 +            for (Term value : terms)
 +                buffers.add(value.bindAndGet(options));
 +            return buffers;
 +        }
 +
 +        /**
 +         * Copies the keys and value from the first <code>Contains</code> to the second one.
 +         *
 +         * @param from the <code>Contains</code> to copy from
 +         * @param to the <code>Contains</code> to copy to
 +         */
 +        private static void copyKeysAndValues(Contains from, Contains to)
 +        {
 +            to.values.addAll(from.values);
 +            to.keys.addAll(from.keys);
 +            to.entryKeys.addAll(from.entryKeys);
 +            to.entryValues.addAll(from.entryValues);
 +        }
 +
 +        private Contains(ColumnDefinition columnDef)
 +        {
 +            super(columnDef);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d0cf7db/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
index 39412b9,98dda26..84343a7
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@@ -21,8 -22,8 +21,6 @@@ import org.junit.Test
  
  import org.apache.cassandra.cql3.CQLTester;
  
--import static org.junit.Assert.assertEquals;
--
  public class SelectMultiColumnRelationTest extends CQLTester
  {
      @Test
@@@ -37,7 -38,9 +35,9 @@@
                                   "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",
 +            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
                                   "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
          }
      }
@@@ -78,40 -81,43 +78,39 @@@
                                   "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
  
              // Wrong type for 'd'
 -            assertInvalidMessage("Expected 4 or 0 byte int (6)",
 -                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
 -
 -            assertInvalidMessage("Invalid tuple type literal for b of type int",
 -                                 "SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
 +            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
 +            assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
  
              // Mix single and tuple inequalities
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                  "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: c < ?",
++             assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                  "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c, d) < (?, ?, ?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
  
 -            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
 +            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
                                   "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
 +            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
                                   "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
 +
 +            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                   "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
  
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
 +            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                   "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
              assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                   "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
-             assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (c) < (?)",
++
++            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
-             assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
 +            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                   "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
++            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                  "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND c < ? AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
  
-             assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (c) < (?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
 -
 -            assertInvalidMessage("PRIMARY KEY column \"d\" cannot be restricted (preceding column \"c\" is restricted by an IN tuple notation)",
 -                                 "SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d0cf7db/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
index 57b8a86,9896d0a..f1bce45
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@@ -514,50 -137,8 +514,78 @@@ public class SelectSingleColumnRelation
  
          assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
                     row(0, 0, 1, 1, 1, 5));
 +    }
 +
 +    @Test
 +    public void testFunctionCallWithUnset() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
 +
 +        assertInvalidMessage("Invalid unset value for argument in call to function token",
 +                             "SELECT * FROM %s WHERE token(k) >= token(?)", unset());
 +        assertInvalidMessage("Invalid unset value for argument in call to function blobasint",
 +                             "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
 +    }
 +
 +    @Test
 +    public void testLimitWithUnset() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
 +        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
 +        execute("INSERT INTO %s (k, i) VALUES (2, 1)");
 +        assertRows(execute("SELECT k FROM %s LIMIT ?", unset()), // treat as 'unlimited'
 +                row(1),
 +                row(2)
 +        );
 +    }
 +
 +    @Test
 +    public void testWithUnsetValues() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (k int, i int, j int, s text, PRIMARY KEY(k,i,j))");
 +        createIndex("CREATE INDEX s_index ON %s (s)");
 +        // partition key
 +        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k = ?", unset());
 +        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN ?", unset());
 +        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?)", unset());
 +        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?,?)", 1, unset());
 +        // clustering column
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i = ?", unset());
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN ?", unset());
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?)", unset());
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?,?)", 1, unset());
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE i = ? ALLOW FILTERING", unset());
 +        // indexed column
 +        assertInvalidMessage("Unsupported unset value for indexed column s", "SELECT * from %s WHERE s = ?", unset());
 +        // range
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i > ?", unset());
 +    }
+ 
 -        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
 -                             "SELECT * FROM %s WHERE a = ? AND d >= ? AND f = ?", 0, 1, 5);
++    @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 but was applied to: 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 but was applied to: a",
++                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (a, b) < (4, 1)");
++        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, b) < (4, 1)");
++        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
++                             "SELECT * FROM %s WHERE b >= 1 and (a, b) < (4, 1)");
++        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
++                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (b) < (4)");
++        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
++                             "SELECT * FROM %s WHERE (b) < (4) and (a, b) >= (1, 1)");
++        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
++                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and a = 1");
+     }
  }


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

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


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

Branch: refs/heads/cassandra-2.2
Commit: 1ae58ef3acc187a3728ca0dda3e98be8b3580ec1
Parents: 07a7e80 de84a5c
Author: blerer <be...@datastax.com>
Authored: Sun Aug 9 22:04:32 2015 +0200
Committer: blerer <be...@datastax.com>
Committed: Sun Aug 9 22:04:56 2015 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



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

Posted by bl...@apache.org.
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-2.2
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))");