You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2014/02/17 10:30:36 UTC

git commit: CQL3: improve support for paginating over composites

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 ea28d3689 -> 652ec6a5c


CQL3: improve support for paginating over composites

patch by slebresne; reviewed by iamaleksey for CASSANDRA-4851


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

Branch: refs/heads/cassandra-2.0
Commit: 652ec6a5c36feae346c71f0ff009ec3b8457448b
Parents: ea28d36
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jan 30 16:11:35 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Mon Feb 17 10:30:29 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 doc/cql3/CQL.textile                            | 24 +++++-
 .../org/apache/cassandra/cql3/CFDefinition.java | 10 +--
 .../cassandra/cql3/ColumnNameBuilder.java       | 11 +--
 src/java/org/apache/cassandra/cql3/Cql.g        | 16 ++++
 .../apache/cassandra/cql3/QueryProcessor.java   |  2 +-
 .../org/apache/cassandra/cql3/Relation.java     | 17 +++-
 .../cassandra/cql3/statements/Restriction.java  | 24 +++++-
 .../cql3/statements/SelectStatement.java        | 82 +++++++++++++++-----
 .../cassandra/db/marshal/CompositeType.java     | 61 +++++++--------
 10 files changed, 175 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 571b8dd..fd3b1b7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -12,6 +12,7 @@
  * Stop CommitLogSegment.close() from calling sync() (CASSANDRA-6652)
  * Make commitlog failure handling configurable (CASSANDRA-6364)
  * Avoid overlaps in LCS (CASSANDRA-6688)
+ * improve support for paginating over composites (4851)
 Merged from 1.2:
  * Fix broken streams when replacing with same IP (CASSANDRA-6622)
  * Fix upgradesstables NPE for non-CF-based indexes (CASSANDRA-6645)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index f82fc19..03b95e0 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1,6 +1,6 @@
 <link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">
 
-h1. Cassandra Query Language (CQL) v3.1.4
+h1. Cassandra Query Language (CQL) v3.1.5
 
 
  <span id="tableOfContents">
@@ -619,10 +619,12 @@ bc(syntax)..
 
 <where-clause> ::= <relation> ( AND <relation> )*
 
-<relation> ::= <identifier> ('=' | '<' | '>' | '<=' | '>=') <term>
+<relation> ::= <identifier> <op> <term>
+             | '(' <identifier> (',' <identifier>)* ')' <op> '(' <term> (',' <term>)* ')'
              | <identifier> IN '(' ( <term> ( ',' <term>)* )? ')'
-             | TOKEN '(' <identifier> ( ',' <identifer>)* ')' ('=' | '<' | '>' | '<=' | '>=') <term>
+             | TOKEN '(' <identifier> ( ',' <identifer>)* ')' <op> <term>
 
+<op> ::= '=' | '<' | '>' | '<=' | '>='
 <order-by> ::= <ordering> ( ',' <odering> )*
 <ordering> ::= <identifer> ( ASC | DESC )?
 p. 
@@ -676,7 +678,7 @@ CREATE TABLE posts (
 The following query is allowed:
 
 bc(sample). 
-SELECT entry_title, content FROM posts WHERE userid='john doe' AND blog_title='John's Blog' AND posted_at >= '2012-01-01' AND posted_at < '2012-01-31'
+SELECT entry_title, content FROM posts WHERE userid='john doe' AND blog_title='John''s Blog' AND posted_at >= '2012-01-01' AND posted_at < '2012-01-31'
 
 But the following one is not, as it does not select a contiguous set of rows (and we suppose no secondary indexes are set):
 
@@ -691,6 +693,16 @@ SELECT * FROM posts WHERE token(userid) > token('tom') AND token(userid) < token
 
 Moreover, the @IN@ relation is only allowed on the last column of the partition key and on the last column of the full primary key.
 
+It is also possible to "group" @CLUSTERING COLUMNS@ together in a relation, for instance:
+
+bc(sample). 
+SELECT * FROM posts WHERE userid='john doe' AND (blog_title, posted_at) > ('John''s Blog', '2012-01-01')
+
+will request all rows that sorts after the one having "John's Blog" as @blog_tile@ and '2012-01-01' for @posted_at@ in the clustering order. In particular, rows having a @post_at <= '2012-01-01'@ will be returned as long as their @blog_title > 'John''s Blog'@, which wouldn't be the case for:
+
+bc(sample). 
+SELECT * FROM posts WHERE userid='john doe' AND blog_title > 'John''s Blog' AND posted_at > '2012-01-01'
+
 h4(#selectOrderBy). @<order-by>@
 
 The @ORDER BY@ option allows to select the order of the returned results. It takes as argument a list of column names along with the order for the column (@ASC@ for ascendant and @DESC@ for descendant, omitting the order being equivalent to @ASC@). Currently the possible orderings are limited (which depends on the table "@CLUSTERING ORDER@":#createTableOptions):
@@ -1101,6 +1113,10 @@ h2(#changes). Changes
 
 The following describes the addition/changes brought for each version of CQL.
 
+h3. 3.1.5
+
+* It is now possible to group clustering columns in a relatiion, see "SELECT Where clauses":#selectWhere.
+
 h3. 3.1.4
 
 * @CREATE INDEX@ now allows specifying options when creating CUSTOM indexes (see "CREATE INDEX reference":#createIndexStmt).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/src/java/org/apache/cassandra/cql3/CFDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CFDefinition.java b/src/java/org/apache/cassandra/cql3/CFDefinition.java
index 54ca2b8..638770d 100644
--- a/src/java/org/apache/cassandra/cql3/CFDefinition.java
+++ b/src/java/org/apache/cassandra/cql3/CFDefinition.java
@@ -246,11 +246,6 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
             return this;
         }
 
-        public NonCompositeBuilder add(ByteBuffer bb, Relation.Type op)
-        {
-            return add(bb);
-        }
-
         public int componentCount()
         {
             return columnName == null ? 0 : 1;
@@ -279,6 +274,11 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
             return build();
         }
 
+        public ByteBuffer buildForRelation(Relation.Type op)
+        {
+            return build();
+        }
+
         public NonCompositeBuilder copy()
         {
             NonCompositeBuilder newBuilder = new NonCompositeBuilder(type);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java b/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java
index b6625ab..3d5eff6 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java
@@ -33,15 +33,6 @@ public interface ColumnNameBuilder
     public ColumnNameBuilder add(ByteBuffer bb);
 
     /**
-     * Add a new ByteBuffer as the next component for this name.
-     * @param t the ByteBuffer to add
-     * @param op the relationship this component should respect.
-     * @throws IllegalStateException if the builder if full, i.e. if enough component has been added.
-     * @return this builder
-     */
-    public ColumnNameBuilder add(ByteBuffer t, Relation.Type op);
-
-    /**
      * Returns the number of component already added to this builder.
      * @return the number of component in this Builder
      */
@@ -70,6 +61,8 @@ public interface ColumnNameBuilder
      */
     public ByteBuffer buildAsEndOfRange();
 
+    public ByteBuffer buildForRelation(Relation.Type op);
+
     /**
      * Clone this builder.
      * @return the cloned builder.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 53aebe7..6e7cf1c 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -881,6 +881,22 @@ relation[List<Relation> clauses]
         { $clauses.add(new Relation(name, Relation.Type.IN, marker)); }
     | name=cident K_IN { Relation rel = Relation.createInRelation($name.id); }
        '(' ( f1=term { rel.addInValue(f1); } (',' fN=term { rel.addInValue(fN); } )* )? ')' { $clauses.add(rel); }
+    | {
+         List<ColumnIdentifier> ids = new ArrayList<ColumnIdentifier>();
+         List<Term.Raw> terms = new ArrayList<Term.Raw>();
+      }
+        '(' n1=cident { ids.add(n1); } (',' ni=cident { ids.add(ni); })* ')'
+        type=relationType
+        '(' t1=term { terms.add(t1); } (',' ti=term { terms.add(ti); })* ')'
+      {
+          if (type == Relation.Type.IN)
+              addRecognitionError("Cannot use IN relation with tuple notation");
+          if (ids.size() != terms.size())
+              addRecognitionError(String.format("Number of values (" + terms.size() + ") in tuple notation doesn't match the number of column names (" + ids.size() + ")"));
+          else
+              for (int i = 0; i < ids.size(); i++)
+                  $clauses.add(new Relation(ids.get(i), type, terms.get(i), i == 0 ? null : ids.get(i-1)));
+      }
     | '(' relation[$clauses] ')'
     ;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 94c6da2..167533f 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -44,7 +44,7 @@ import org.apache.cassandra.utils.SemanticVersion;
 
 public class QueryProcessor
 {
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.1.4");
+    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.1.5");
 
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
     private static final MemoryMeter meter = new MemoryMeter();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index 15ed540..9d065bf 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -33,18 +33,22 @@ public class Relation
     private final List<Term.Raw> inValues;
     public final boolean onToken;
 
+    // Will be null unless for tuple notations (#4851)
+    public final ColumnIdentifier previousInTuple;
+
     public static enum Type
     {
         EQ, LT, LTE, GTE, GT, IN;
     }
 
-    private Relation(ColumnIdentifier entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
+    private Relation(ColumnIdentifier entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken, ColumnIdentifier previousInTuple)
     {
         this.entity = entity;
         this.relationType = type;
         this.value = value;
         this.inValues = inValues;
         this.onToken = onToken;
+        this.previousInTuple = previousInTuple;
     }
 
     /**
@@ -56,17 +60,22 @@ public class Relation
      */
     public Relation(ColumnIdentifier entity, Type type, Term.Raw value)
     {
-        this(entity, type, value, null, false);
+        this(entity, type, value, null, false, null);
     }
 
     public Relation(ColumnIdentifier entity, Type type, Term.Raw value, boolean onToken)
     {
-        this(entity, type, value, null, onToken);
+        this(entity, type, value, null, onToken, null);
+    }
+
+    public Relation(ColumnIdentifier entity, Type type, Term.Raw value, ColumnIdentifier previousInTuple)
+    {
+        this(entity, type, value, null, false, previousInTuple);
     }
 
     public static Relation createInRelation(ColumnIdentifier entity)
     {
-        return new Relation(entity, Type.IN, null, new ArrayList<Term.Raw>(), false);
+        return new Relation(entity, Type.IN, null, new ArrayList<Term.Raw>(), false, null);
     }
 
     public Type operator()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/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 3a3aa05..6323acb 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@ -22,6 +22,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.base.Objects;
+
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.thrift.IndexOperator;
 import org.apache.cassandra.cql3.*;
@@ -188,10 +190,16 @@ public interface Restriction
         private final boolean[] boundInclusive;
         private final boolean onToken;
 
+        // The name of the column that was preceding this one if the tuple notation of #4851 was used
+        // (note: if it is set for both bound, we'll validate both have the same previous value, but we
+        // still need to distinguish if it's set or not for both bound)
+        private final ColumnIdentifier[] previous;
+
         public Slice(boolean onToken)
         {
             this.bounds = new Term[2];
             this.boundInclusive = new boolean[2];
+            this.previous = new ColumnIdentifier[2];
             this.onToken = onToken;
         }
 
@@ -259,7 +267,7 @@ public interface Restriction
             throw new AssertionError();
         }
 
-        public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException
+        public void setBound(ColumnIdentifier name, Relation.Type type, Term t, ColumnIdentifier previousName) throws InvalidRequestException
         {
             Bound b;
             boolean inclusive;
@@ -290,6 +298,20 @@ public interface Restriction
 
             bounds[b.idx] = t;
             boundInclusive[b.idx] = inclusive;
+
+            // If a bound is part of a tuple notation (#4851), the other bound must either also be or must not be set at all,
+            // and this even if there is a 2ndary index (it's not supported by the 2ndary code). And it's easier to validate
+            // this here so we do.
+            Bound reverse = Bound.reverse(b);
+            if (hasBound(reverse) && !(Objects.equal(previousName, previous[reverse.idx])))
+                throw new InvalidRequestException(String.format("Clustering column %s cannot be restricted both inside a tuple notation and outside it", name));
+
+            previous[b.idx] = previousName;
+        }
+
+        public boolean isPartOfTuple()
+        {
+            return previous[Bound.START.idx] != null || previous[Bound.END.idx] != null;
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/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 307e668..d42fd76 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -667,14 +667,16 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         // to the component comparator but not to the end-of-component itself),
         // it only depends on whether the slice is reversed
         Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-        for (CFDefinition.Name name : names)
+        for (Iterator<CFDefinition.Name> iter = names.iterator(); iter.hasNext();)
         {
+            CFDefinition.Name name = iter.next();
+
             // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
             // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
             // But if the actual comparator itself is reversed, we must inversed the bounds too.
             Bound b = isReversed == isReversedType(name) ? bound : Bound.reverse(bound);
             Restriction r = restrictions[name.position];
-            if (r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b)))
+            if (isNullRestriction(r, b))
             {
                 // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix.
                 // For composites, if there was preceding component and we're computing the end, we must change the last component
@@ -686,12 +688,21 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
             if (r.isSlice())
             {
-                Restriction.Slice slice = (Restriction.Slice)r;
-                assert slice.hasBound(b);
-                ByteBuffer val = slice.bound(b, variables);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
-                return Collections.singletonList(builder.add(val, slice.getRelation(eocBound, b)).build());
+                builder.add(getSliceValue(name, r, b, variables));
+                Relation.Type relType = ((Restriction.Slice)r).getRelation(eocBound, b);
+
+                // We can have more non null restriction if the "scalar" notation was used for the bound (#4851).
+                // In that case, we need to add them all, and end the cell name with the correct end-of-component.
+                while (iter.hasNext())
+                {
+                    name = iter.next();
+                    r = restrictions[name.position];
+                    if (isNullRestriction(r, b))
+                        break;
+
+                    builder.add(getSliceValue(name, r, b, variables));
+                }
+                return Collections.singletonList(builder.buildForRelation(relType));
             }
             else
             {
@@ -729,6 +740,21 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return Collections.singletonList((bound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
     }
 
+    private static boolean isNullRestriction(Restriction r, Bound b)
+    {
+        return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
+    }
+
+    private static ByteBuffer getSliceValue(CFDefinition.Name name, Restriction r, Bound b, List<ByteBuffer> variables) throws InvalidRequestException
+    {
+        Restriction.Slice slice = (Restriction.Slice)r;
+        assert slice.hasBound(b);
+        ByteBuffer val = slice.bound(b, variables);
+        if (val == null)
+            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
+        return val;
+    }
+
     private List<ByteBuffer> getRequestedBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
     {
         assert isColumnRange();
@@ -794,7 +820,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return expressions;
     }
 
-
     private Iterable<Column> columnsInOrder(final ColumnFamily cf, final List<ByteBuffer> variables) throws InvalidRequestException
     {
         if (columnRestrictions.length == 0)
@@ -1139,16 +1164,16 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 switch (name.kind)
                 {
                     case KEY_ALIAS:
-                        stmt.keyRestrictions[name.position] = updateRestriction(name, stmt.keyRestrictions[name.position], rel, names);
+                        stmt.keyRestrictions[name.position] = updateRestriction(cfm, name, stmt.keyRestrictions[name.position], rel, names);
                         break;
                     case COLUMN_ALIAS:
-                        stmt.columnRestrictions[name.position] = updateRestriction(name, stmt.columnRestrictions[name.position], rel, names);
+                        stmt.columnRestrictions[name.position] = updateRestriction(cfm, name, stmt.columnRestrictions[name.position], rel, 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:
                         // 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 = updateRestriction(name, stmt.metadataRestrictions.get(name), rel, names);
+                        Restriction r = updateRestriction(cfm, name, stmt.metadataRestrictions.get(name), rel, 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));
@@ -1229,6 +1254,8 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 {
                     // Non EQ relation is not supported without token(), even if we have a 2ndary index (since even those are ordered by partitioner).
                     // Note: In theory we could allow it for 2ndary index queries with ALLOW FILTERING, but that would probably require some special casing
+                    // Note bis: This is also why we don't bother handling the 'tuple' notation of #4851 for keys. If we lift the limitation for 2ndary
+                    // index with filtering, we'll need to handle it though.
                     throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
                 }
                 previous = cname;
@@ -1244,6 +1271,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             // the column is indexed that is.
             canRestrictFurtherComponents = true;
             previous = null;
+            boolean previousIsSlice = false;
             iter = cfDef.columns.values().iterator();
             for (int i = 0; i < stmt.columnRestrictions.length; i++)
             {
@@ -1253,19 +1281,31 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 if (restriction == null)
                 {
                     canRestrictFurtherComponents = false;
+                    previousIsSlice = false;
                 }
                 else if (!canRestrictFurtherComponents)
                 {
-                    if (hasQueriableIndex)
+                    // We're here if the previous clustering column was either not restricted or was a slice.
+                    // We can't restrict the current column unless:
+                    //   1) we're in the special case of the 'tuple' notation from #4851 which we expand as multiple
+                    //      consecutive slices: in which case we're good with this restriction and we continue
+                    //   2) we have a 2ndary index, in which case we have to use it but can skip more validation
+                    boolean hasTuple = false;
+                    boolean hasRestrictedNotTuple = false;
+                    if (!(previousIsSlice && restriction.isSlice() && ((Restriction.Slice)restriction).isPartOfTuple()))
                     {
-                        stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
-                        break;
+                        if (hasQueriableIndex)
+                        {
+                            stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
+                            break;
+                        }
+                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
                     }
-                    throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
                 }
                 else if (restriction.isSlice())
                 {
                     canRestrictFurtherComponents = false;
+                    previousIsSlice = true;
                     Restriction.Slice slice = (Restriction.Slice)restriction;
                     // For non-composite slices, we don't support internally the difference between exclusive and
                     // inclusive bounds, so we deal with it manually.
@@ -1446,7 +1486,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
         }
 
-        Restriction updateRestriction(CFDefinition.Name name, Restriction restriction, Relation newRel, VariableSpecifications boundNames) throws InvalidRequestException
+        Restriction updateRestriction(CFMetaData cfm, CFDefinition.Name name, Restriction restriction, Relation newRel, VariableSpecifications boundNames) throws InvalidRequestException
         {
             ColumnSpecification receiver = name;
             if (newRel.onToken)
@@ -1460,6 +1500,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                                                    StorageService.getPartitioner().getTokenValidator());
             }
 
+            // We can only use the tuple notation of #4851 on clustering columns for now
+            if (newRel.previousInTuple != null && name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
+                throw new InvalidRequestException(String.format("Tuple notation can only be used on clustering columns but found on %s", name));
+
             switch (newRel.operator())
             {
                 case EQ:
@@ -1506,7 +1550,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                             throw new InvalidRequestException(String.format("%s cannot be restricted by both an equal and an inequal relation", name));
                         Term t = newRel.getValue().prepare(receiver);
                         t.collectMarkerSpecification(boundNames);
-                        ((Restriction.Slice)restriction).setBound(name.name, newRel.operator(), t);
+                        if (newRel.previousInTuple != null && (name.position == 0 || !cfm.clusteringKeyColumns().get(name.position - 1).name.equals(newRel.previousInTuple.key)))
+                            throw new InvalidRequestException(String.format("Invalid tuple notation, column %s is not before column %s in the clustering order", newRel.previousInTuple, name.name));
+                        ((Restriction.Slice)restriction).setBound(name.name, newRel.operator(), t, newRel.previousInTuple);
                     }
                     break;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/652ec6a5/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index a44b88f..2c0e121 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -338,44 +338,15 @@ public class CompositeType extends AbstractCompositeType
             this.serializedSize = b.serializedSize;
         }
 
-        public Builder add(ByteBuffer buffer, Relation.Type op)
+        public Builder add(ByteBuffer bb)
         {
             if (components.size() >= composite.types.size())
                 throw new IllegalStateException("Composite column is already fully constructed");
 
-            int current = components.size();
-            components.add(buffer);
-
-            /*
-             * Given the rules for eoc (end-of-component, see AbstractCompositeType.compare()),
-             * We can select:
-             *   - = 'a' by using <'a'><0>
-             *   - < 'a' by using <'a'><-1>
-             *   - <= 'a' by using <'a'><1>
-             *   - > 'a' by using <'a'><1>
-             *   - >= 'a' by using <'a'><0>
-             */
-            switch (op)
-            {
-                case LT:
-                    endOfComponents[current] = (byte) -1;
-                    break;
-                case GT:
-                case LTE:
-                    endOfComponents[current] = (byte) 1;
-                    break;
-                default:
-                    endOfComponents[current] = (byte) 0;
-                    break;
-            }
+            components.add(bb);
             return this;
         }
 
-        public Builder add(ByteBuffer bb)
-        {
-            return add(bb, Relation.Type.EQ);
-        }
-
         public int componentCount()
         {
             return components.size();
@@ -419,6 +390,34 @@ public class CompositeType extends AbstractCompositeType
             return bb;
         }
 
+        public ByteBuffer buildForRelation(Relation.Type op)
+        {
+            /*
+             * Given the rules for eoc (end-of-component, see AbstractCompositeType.compare()),
+             * We can select:
+             *   - = 'a' by using <'a'><0>
+             *   - < 'a' by using <'a'><-1>
+             *   - <= 'a' by using <'a'><1>
+             *   - > 'a' by using <'a'><1>
+             *   - >= 'a' by using <'a'><0>
+             */
+            int current = components.size() - 1;
+            switch (op)
+            {
+                case LT:
+                    endOfComponents[current] = (byte) -1;
+                    break;
+                case GT:
+                case LTE:
+                    endOfComponents[current] = (byte) 1;
+                    break;
+                default:
+                    endOfComponents[current] = (byte) 0;
+                    break;
+            }
+            return build();
+        }
+
         public Builder copy()
         {
             return new Builder(this);