Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7C97910651 for ; Mon, 17 Feb 2014 14:04:40 +0000 (UTC) Received: (qmail 68543 invoked by uid 500); 17 Feb 2014 14:03:01 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 68459 invoked by uid 500); 17 Feb 2014 14:02:58 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 68245 invoked by uid 99); 17 Feb 2014 14:02:49 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 17 Feb 2014 14:02:49 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 833178B29CB; Mon, 17 Feb 2014 14:02:49 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: slebresne@apache.org To: commits@cassandra.apache.org Date: Mon, 17 Feb 2014 14:02:49 -0000 Message-Id: <532f6555cdb94dc08c7a00f6dbc8a474@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/3] git commit: CQL3: improve support for paginating over composites Repository: cassandra Updated Branches: refs/heads/trunk 9ea99491e -> 4c727f6f9 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/trunk Commit: 652ec6a5c36feae346c71f0ff009ec3b8457448b Parents: ea28d36 Author: Sylvain Lebresne Authored: Thu Jan 30 16:11:35 2014 +0100 Committer: Sylvain Lebresne 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 @@ -h1. Cassandra Query Language (CQL) v3.1.4 +h1. Cassandra Query Language (CQL) v3.1.5 @@ -619,10 +619,12 @@ bc(syntax).. ::= ( AND )* - ::= ('=' | '<' | '>' | '<=' | '>=') + ::= + | '(' (',' )* ')' '(' (',' )* ')' | IN '(' ( ( ',' )* )? ')' - | TOKEN '(' ( ',' )* ')' ('=' | '<' | '>' | '<=' | '>=') + | TOKEN '(' ( ',' )* ')' + ::= '=' | '<' | '>' | '<=' | '>=' ::= ( ',' )* ::= ( 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). @@ 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 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 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 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 ids = new ArrayList(); + List terms = new ArrayList(); + } + '(' 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 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 inValues, boolean onToken) + private Relation(ColumnIdentifier entity, Type type, Term.Raw value, List 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(), false); + return new Relation(entity, Type.IN, null, new ArrayList(), 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 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 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 getRequestedBound(Bound b, List variables) throws InvalidRequestException { assert isColumnRange(); @@ -794,7 +820,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache return expressions; } - private Iterable columnsInOrder(final ColumnFamily cf, final List 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);