You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/05/22 21:20:25 UTC
[5/6] Merge branch 'cassandra-2.0' into cassandra-2.1
http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 420f475,e339ccb..1f9688a
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -69,12 -70,17 +69,17 @@@ public class SelectStatement implement
private final Selection selection;
private final Term limit;
+ /** Restrictions on partitioning columns */
private final Restriction[] keyRestrictions;
+
+ /** Restrictions on clustering columns */
private final Restriction[] columnRestrictions;
+
+ /** Restrictions on non-primary key columns (i.e. secondary index restrictions) */
- private final Map<CFDefinition.Name, Restriction> metadataRestrictions = new HashMap<CFDefinition.Name, Restriction>();
+ private final Map<ColumnIdentifier, Restriction> metadataRestrictions = new HashMap<ColumnIdentifier, Restriction>();
- // The name of all restricted names not covered by the key or index filter
- private final Set<CFDefinition.Name> restrictedNames = new HashSet<CFDefinition.Name>();
+ // All restricted columns not covered by the key or index filter
+ private final Set<ColumnDefinition> restrictedColumns = new HashSet<ColumnDefinition>();
private Restriction.Slice sliceRestriction;
private boolean isReversed;
@@@ -327,9 -335,9 +332,9 @@@
if (keys.isEmpty()) // in case of IN () for (the last column of) the partition key.
return null;
- List<ReadCommand> commands = new ArrayList<ReadCommand>(keys.size());
+ List<ReadCommand> commands = new ArrayList<>(keys.size());
- IDiskAtomFilter filter = makeFilter(variables, limit);
+ IDiskAtomFilter filter = makeFilter(options, limit);
if (filter == null)
return null;
@@@ -765,15 -794,29 +777,30 @@@
return false;
}
- private List<ByteBuffer> buildBound(Bound bound,
- Collection<CFDefinition.Name> names,
- Restriction[] restrictions,
- boolean isReversed,
- ColumnNameBuilder builder,
- List<ByteBuffer> variables) throws InvalidRequestException
+ private static List<Composite> buildBound(Bound bound,
+ Collection<ColumnDefinition> defs,
+ Restriction[] restrictions,
+ boolean isReversed,
+ CType type,
+ QueryOptions options) throws InvalidRequestException
{
+ CBuilder builder = type.builder();
+ // check the first restriction to see if we're dealing with a multi-column restriction
- if (!names.isEmpty())
++ if (!defs.isEmpty())
+ {
+ Restriction firstRestriction = restrictions[0];
+ if (firstRestriction != null && firstRestriction.isMultiColumn())
+ {
+ if (firstRestriction.isSlice())
- return buildMultiColumnSliceBound(bound, names, (MultiColumnRestriction.Slice) firstRestriction, isReversed, builder, variables);
++ return buildMultiColumnSliceBound(bound, defs, (MultiColumnRestriction.Slice) firstRestriction, isReversed, builder, options);
+ else if (firstRestriction.isIN())
- return buildMultiColumnInBound(bound, names, (MultiColumnRestriction.IN) firstRestriction, isReversed, builder, variables);
++ return buildMultiColumnInBound(bound, defs, (MultiColumnRestriction.IN) firstRestriction, isReversed, builder, type, options);
+ else
- return buildMultiColumnEQBound(bound, (MultiColumnRestriction.EQ) firstRestriction, isReversed, builder, variables);
++ return buildMultiColumnEQBound(bound, (MultiColumnRestriction.EQ) firstRestriction, isReversed, builder, options);
+ }
+ }
+
// The end-of-component of composite doesn't depend on whether the
// component type is reversed or not (i.e. the ReversedType is applied
// to the component comparator but not to the end-of-component itself),
@@@ -793,47 -836,35 +820,34 @@@
// 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
// End-Of-Component, otherwise we would be selecting only one record.
- return Collections.singletonList(builder.componentCount() > 0 && eocBound == Bound.END
- ? builder.buildAsEndOfRange()
- : builder.build());
+ Composite prefix = builder.build();
+ return Collections.singletonList(!prefix.isEmpty() && eocBound == Bound.END ? prefix.end() : prefix);
}
-
if (r.isSlice())
{
- builder.add(getSliceValue(def, r, b, options));
- builder.add(getSliceValue(r, b, variables));
++ builder.add(getSliceValue(r, b, options));
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())
- {
- def = iter.next();
- r = restrictions[def.position()];
- if (isNullRestriction(r, b))
- break;
-
- builder.add(getSliceValue(def, r, b, options));
- }
- return Collections.singletonList(builder.buildForRelation(relType));
+ return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
}
else
{
- List<ByteBuffer> values = r.values(variables);
+ List<ByteBuffer> values = r.values(options);
if (values.size() != 1)
{
- // IN query, we only support it on the clustering column
+ // IN query, we only support it on the clustering columns
- assert name.position == names.size() - 1;
+ assert def.position() == defs.size() - 1;
// The IN query might not have listed the values in comparator order, so we need to re-sort
// the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
- TreeSet<Composite> s = new TreeSet<Composite>(isReversed ? type.reverseComparator() : type);
- TreeSet<ByteBuffer> s = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
++ TreeSet<Composite> s = new TreeSet<>(isReversed ? type.reverseComparator() : type);
for (ByteBuffer val : values)
{
if (val == null)
- throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
- ColumnNameBuilder copy = builder.copy().add(val);
+ throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
+ Composite prefix = builder.buildWith(val);
// See below for why this
- s.add((b == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
+ s.add((b == Bound.END && builder.remainingCount() > 0) ? prefix.end() : prefix);
}
- return new ArrayList<Composite>(s);
+ return new ArrayList<>(s);
}
ByteBuffer val = values.get(0);
@@@ -847,42 -878,96 +861,123 @@@
// it would be harmless to do it. However, we use this method got the partition key too. And when a query
// with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
// case using the eoc would be bad, since for the random partitioner we have no guarantee that
- // builder.buildAsEndOfRange() will sort after builder.build() (see #5240).
- return Collections.singletonList((bound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
+ // prefix.end() will sort after prefix (see #5240).
+ Composite prefix = builder.build();
+ return Collections.singletonList(bound == Bound.END && builder.remainingCount() > 0 ? prefix.end() : prefix);
+ }
+
+ private static Composite.EOC eocForRelation(Relation.Type op)
+ {
+ switch (op)
+ {
+ case LT:
+ // < X => using startOf(X) as finish bound
+ return Composite.EOC.START;
+ case GT:
+ case LTE:
+ // > X => using endOf(X) as start bound
+ // <= X => using endOf(X) as finish bound
+ return Composite.EOC.END;
+ default:
+ // >= X => using X as start bound (could use START_OF too)
+ // = X => using X
+ return Composite.EOC.NONE;
+ }
}
- private List<ByteBuffer> buildMultiColumnSliceBound(Bound bound,
- Collection<CFDefinition.Name> names,
- MultiColumnRestriction.Slice slice,
- boolean isReversed,
- ColumnNameBuilder builder,
- List<ByteBuffer> variables) throws InvalidRequestException
++ private static List<Composite> buildMultiColumnSliceBound(Bound bound,
++ Collection<ColumnDefinition> defs,
++ MultiColumnRestriction.Slice slice,
++ boolean isReversed,
++ CBuilder builder,
++ QueryOptions options) throws InvalidRequestException
+ {
+ Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
+
- Iterator<CFDefinition.Name> iter = names.iterator();
- CFDefinition.Name firstName = iter.next();
++ Iterator<ColumnDefinition> iter = defs.iterator();
++ ColumnDefinition firstName = iter.next();
+ // A hack to preserve pre-6875 behavior for tuple-notation slices where the comparator mixes ASCENDING
+ // and DESCENDING orders. This stores the bound for the first component; we will re-use it for all following
+ // components, even if they don't match the first component's reversal/non-reversal. Note that this does *not*
+ // guarantee correct query results, it just preserves the previous behavior.
+ Bound firstComponentBound = isReversed == isReversedType(firstName) ? bound : Bound.reverse(bound);
+
+ if (!slice.hasBound(firstComponentBound))
- return Collections.singletonList(builder.componentCount() > 0 && eocBound == Bound.END
- ? builder.buildAsEndOfRange()
- : builder.build());
++ {
++ Composite prefix = builder.build();
++ return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
++ ? prefix.end()
++ : prefix);
++ }
+
- List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, variables);
- builder.add(vals.get(firstName.position));
++ List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, options);
++ builder.add(vals.get(firstName.position()));
+
+ while(iter.hasNext())
+ {
- CFDefinition.Name name = iter.next();
- if (name.position >= vals.size())
++ ColumnDefinition def = iter.next();
++ if (def.position() >= vals.size())
+ break;
+
- builder.add(vals.get(name.position));
++ builder.add(vals.get(def.position()));
+ }
+ Relation.Type relType = slice.getRelation(eocBound, firstComponentBound);
- return Collections.singletonList(builder.buildForRelation(relType));
++ return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
+ }
+
- private List<ByteBuffer> buildMultiColumnInBound(Bound bound,
- Collection<CFDefinition.Name> names,
- MultiColumnRestriction.IN restriction,
- boolean isReversed,
- ColumnNameBuilder builder,
- List<ByteBuffer> variables) throws InvalidRequestException
++ private static List<Composite> buildMultiColumnInBound(Bound bound,
++ Collection<ColumnDefinition> defs,
++ MultiColumnRestriction.IN restriction,
++ boolean isReversed,
++ CBuilder builder,
++ CType type,
++ QueryOptions options) throws InvalidRequestException
+ {
- List<List<ByteBuffer>> splitInValues = restriction.splitValues(variables);
++ List<List<ByteBuffer>> splitInValues = restriction.splitValues(options);
+
+ // The IN query might not have listed the values in comparator order, so we need to re-sort
+ // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
- TreeSet<ByteBuffer> inValues = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
- Iterator<CFDefinition.Name> iter = names.iterator();
++ TreeSet<Composite> inValues = new TreeSet<>(isReversed ? type.reverseComparator() : type);
++ Iterator<ColumnDefinition> iter = defs.iterator();
+ for (List<ByteBuffer> components : splitInValues)
+ {
- ColumnNameBuilder nameBuilder = builder.copy();
- for (ByteBuffer component : components)
- nameBuilder.add(component);
-
++ Composite prefix = builder.buildWith(components);
+ Bound b = isReversed == isReversedType(iter.next()) ? bound : Bound.reverse(bound);
- inValues.add((bound == Bound.END && nameBuilder.remainingCount() > 0) ? nameBuilder.buildAsEndOfRange() : nameBuilder.build());
++ inValues.add(b == Bound.END && builder.remainingCount() - components.size() > 0
++ ? prefix.end()
++ : prefix);
+ }
+ return new ArrayList<>(inValues);
+ }
+
- private List<ByteBuffer> buildMultiColumnEQBound(Bound bound, MultiColumnRestriction.EQ restriction, boolean isReversed, ColumnNameBuilder builder, List<ByteBuffer> variables) throws InvalidRequestException
++ private static List<Composite> buildMultiColumnEQBound(Bound bound,
++ MultiColumnRestriction.EQ restriction,
++ boolean isReversed,
++ CBuilder builder,
++ QueryOptions options) throws InvalidRequestException
+ {
+ Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
- for (ByteBuffer component : restriction.values(variables))
++ for (ByteBuffer component : restriction.values(options))
+ builder.add(component);
+
- ByteBuffer result = builder.componentCount() > 0 && eocBound == Bound.END
- ? builder.buildAsEndOfRange()
- : builder.build();
- return Collections.singletonList(result);
++ Composite prefix = builder.build();
++ return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
++ ? prefix.end()
++ : prefix);
+ }
+
private static boolean isNullRestriction(Restriction r, Bound b)
{
return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
}
- private static ByteBuffer getSliceValue(ColumnDefinition def, Restriction r, Bound b, QueryOptions options) throws InvalidRequestException
- private static ByteBuffer getSliceValue(Restriction r, Bound b, List<ByteBuffer> variables) throws InvalidRequestException
++ private static ByteBuffer getSliceValue(Restriction r, Bound b, QueryOptions options) throws InvalidRequestException
{
Restriction.Slice slice = (Restriction.Slice)r;
assert slice.hasBound(b);
- ByteBuffer val = slice.bound(b, variables);
+ ByteBuffer val = slice.bound(b, options);
if (val == null)
- throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
+ throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
return val;
}
@@@ -936,23 -1022,9 +1031,23 @@@
}
}
}
+ else if (restriction.isContains())
+ {
- Restriction.Contains contains = (Restriction.Contains)restriction;
++ SingleColumnRestriction.Contains contains = (SingleColumnRestriction.Contains)restriction;
+ for (ByteBuffer value : contains.values(options))
+ {
+ validateIndexedValue(def, value);
+ expressions.add(new IndexExpression(def.name.bytes, IndexExpression.Operator.CONTAINS, value));
+ }
+ for (ByteBuffer key : contains.keys(options))
+ {
+ validateIndexedValue(def, key);
+ expressions.add(new IndexExpression(def.name.bytes, IndexExpression.Operator.CONTAINS_KEY, key));
+ }
+ }
else
{
- List<ByteBuffer> values = restriction.values(variables);
+ List<ByteBuffer> values = restriction.values(options);
if (values.size() != 1)
throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
@@@ -1218,28 -1352,23 +1313,20 @@@
public ParsedStatement.Prepared prepare() throws InvalidRequestException
{
CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
--
- VariableSpecifications names = getBoundVariables();
- CFDefinition cfDef = cfm.getCfDef();
-
+ VariableSpecifications boundNames = getBoundVariables();
// Select clause
if (parameters.isCount && !selectClause.isEmpty())
throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
Selection selection = selectClause.isEmpty()
- ? Selection.wildcard(cfDef)
- : Selection.fromSelectors(cfDef, selectClause);
+ ? Selection.wildcard(cfm)
+ : Selection.fromSelectors(cfm, selectClause);
if (parameters.isDistinct)
- validateDistinctSelection(selection.getColumns(), cfDef.partitionKeys());
+ validateDistinctSelection(selection.getColumns(), cfm.partitionKeyColumns());
- Term prepLimit = null;
- if (limit != null)
- {
- prepLimit = limit.prepare(keyspace(), limitReceiver());
- prepLimit.collectMarkerSpecification(names);
- }
-
- SelectStatement stmt = new SelectStatement(cfm, names.size(), parameters, selection, prepLimit);
- SelectStatement stmt = new SelectStatement(cfDef, boundNames.size(), parameters, selection, prepareLimit(boundNames));
++ SelectStatement stmt = new SelectStatement(cfm, boundNames.size(), parameters, selection, prepareLimit(boundNames));
/*
* WHERE clause. For a given entity, rules are:
@@@ -1251,51 -1380,318 +1338,338 @@@
*/
boolean hasQueriableIndex = false;
boolean hasQueriableClusteringColumnIndex = false;
- for (Relation rel : whereClause)
+ for (Relation relation : whereClause)
{
- ColumnDefinition def = cfm.getColumnDefinition(rel.getEntity());
- if (def == null)
+ if (relation.isMultiColumn())
+ {
+ MultiColumnRelation rel = (MultiColumnRelation) relation;
- List<CFDefinition.Name> names = new ArrayList<>(rel.getEntities().size());
++ List<ColumnDefinition> names = new ArrayList<>(rel.getEntities().size());
+ for (ColumnIdentifier entity : rel.getEntities())
+ {
- boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
++ ColumnDefinition def = cfm.getColumnDefinition(entity);
++ boolean[] queriable = processRelationEntity(stmt, relation, entity, def);
+ hasQueriableIndex |= queriable[0];
+ hasQueriableClusteringColumnIndex |= queriable[1];
- names.add(cfDef.get(entity));
++ names.add(def);
+ }
+ updateRestrictionsForRelation(stmt, names, rel, boundNames);
+ }
+ else
+ {
+ SingleColumnRelation rel = (SingleColumnRelation) relation;
- boolean[] queriable = processRelationEntity(stmt, relation, rel.getEntity(), cfDef);
++ ColumnIdentifier entity = rel.getEntity();
++ ColumnDefinition def = cfm.getColumnDefinition(entity);
++ boolean[] queriable = processRelationEntity(stmt, relation, entity, def);
+ hasQueriableIndex |= queriable[0];
+ hasQueriableClusteringColumnIndex |= queriable[1];
- updateRestrictionsForRelation(stmt, cfDef.get(rel.getEntity()), rel, boundNames);
++ updateRestrictionsForRelation(stmt, def, rel, boundNames);
+ }
+ }
+
+ // At this point, the select statement if fully constructed, but we still have a few things to validate
- processPartitionKeyRestrictions(stmt, cfDef, hasQueriableIndex);
++ processPartitionKeyRestrictions(stmt, hasQueriableIndex, cfm);
+
+ // All (or none) of the partition key columns have been specified;
+ // hence there is no need to turn these restrictions into index expressions.
+ if (!stmt.usesSecondaryIndexing)
- stmt.restrictedNames.removeAll(cfDef.partitionKeys());
++ stmt.restrictedColumns.removeAll(cfm.partitionKeyColumns());
+
+ if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
+ throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
+
- processColumnRestrictions(stmt, cfDef, hasQueriableIndex);
++ processColumnRestrictions(stmt, hasQueriableIndex, cfm);
+
+ // Covers indexes on the first clustering column (among others).
+ if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
+ stmt.usesSecondaryIndexing = true;
+
+ if (!stmt.usesSecondaryIndexing)
- stmt.restrictedNames.removeAll(cfDef.clusteringColumns());
++ stmt.restrictedColumns.removeAll(cfm.clusteringColumns());
+
+ // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
+ // there is restrictions not covered by the PK.
+ if (!stmt.metadataRestrictions.isEmpty())
+ {
+ if (!hasQueriableIndex)
+ throw new InvalidRequestException("No indexed columns present in by-columns clause with Equal operator");
+ stmt.usesSecondaryIndexing = true;
+ }
+
+ if (stmt.usesSecondaryIndexing)
+ validateSecondaryIndexSelections(stmt);
+
+ if (!stmt.parameters.orderings.isEmpty())
- processOrderingClause(stmt, cfDef);
++ processOrderingClause(stmt, cfm);
+
+ checkNeedsFiltering(stmt);
+
+ return new ParsedStatement.Prepared(stmt, boundNames);
+ }
+
+ /** Returns a pair of (hasQueriableIndex, hasQueriableClusteringColumnIndex) */
- private boolean[] processRelationEntity(SelectStatement stmt, Relation relation, ColumnIdentifier entity, CFDefinition cfDef) throws InvalidRequestException
++ private boolean[] processRelationEntity(SelectStatement stmt, Relation relation, ColumnIdentifier entity, ColumnDefinition def) throws InvalidRequestException
+ {
- CFDefinition.Name name = cfDef.get(entity);
- if (name == null)
++ if (def == null)
+ handleUnrecognizedEntity(entity, relation);
+
- stmt.restrictedNames.add(name);
- if (cfDef.cfm.getColumnDefinition(name.name.key).isIndexed() && relation.operator() == Relation.Type.EQ)
- return new boolean[]{true, name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS};
++ stmt.restrictedColumns.add(def);
++ if (def.isIndexed() && relation.operator().allowsIndexQuery())
++ return new boolean[]{true, def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN};
+ return new boolean[]{false, false};
+ }
+
+ /** Throws an InvalidRequestException for an unrecognized identifier in the WHERE clause */
+ private void handleUnrecognizedEntity(ColumnIdentifier entity, Relation relation) throws InvalidRequestException
+ {
+ if (containsAlias(entity))
+ throw new InvalidRequestException(String.format("Aliases aren't allowed in the where clause ('%s')", relation));
+ else
+ throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", entity, relation));
+ }
+
+ /** Returns a Term for the limit or null if no limit is set */
+ private Term prepareLimit(VariableSpecifications boundNames) throws InvalidRequestException
+ {
+ if (limit == null)
+ return null;
+
- Term prepLimit = limit.prepare(limitReceiver());
++ Term prepLimit = limit.prepare(keyspace(), limitReceiver());
+ prepLimit.collectMarkerSpecification(boundNames);
+ return prepLimit;
+ }
+
- private void updateRestrictionsForRelation(SelectStatement stmt, List<CFDefinition.Name> names, MultiColumnRelation relation, VariableSpecifications boundNames) throws InvalidRequestException
++ private void updateRestrictionsForRelation(SelectStatement stmt, List<ColumnDefinition> defs, MultiColumnRelation relation, VariableSpecifications boundNames) throws InvalidRequestException
+ {
- List<CFDefinition.Name> restrictedColumns = new ArrayList<>();
- Set<CFDefinition.Name> seen = new HashSet<>();
++ List<ColumnDefinition> restrictedColumns = new ArrayList<>();
++ Set<ColumnDefinition> seen = new HashSet<>();
+
+ int previousPosition = -1;
- for (CFDefinition.Name name : names)
++ for (ColumnDefinition def : defs)
+ {
+ // ensure multi-column restriction only applies to clustering columns
- if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
- throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", name));
++ if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
++ throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", def));
+
- if (seen.contains(name))
- throw new InvalidRequestException(String.format("Column \"%s\" appeared twice in a relation: %s", name, relation));
- seen.add(name);
++ if (seen.contains(def))
++ throw new InvalidRequestException(String.format("Column \"%s\" appeared twice in a relation: %s", def, relation));
++ seen.add(def);
+
+ // check that no clustering columns were skipped
- if (name.position != previousPosition + 1)
++ if (def.position() != previousPosition + 1)
{
- if (containsAlias(rel.getEntity()))
- throw new InvalidRequestException(String.format("Aliases aren't allowed in where clause ('%s')", rel));
+ if (previousPosition == -1)
+ throw new InvalidRequestException(String.format(
+ "Clustering columns may not be skipped in multi-column relations. " +
+ "They should appear in the PRIMARY KEY order. Got %s", relation));
else
- throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", rel.getEntity(), rel));
+ throw new InvalidRequestException(String.format(
+ "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
}
+ previousPosition++;
- stmt.restrictedColumns.add(def);
- if (def.isIndexed() && rel.operator().allowsIndexQuery())
- Restriction existing = getExistingRestriction(stmt, name);
++ Restriction existing = getExistingRestriction(stmt, def);
+ Relation.Type operator = relation.operator();
+ if (existing != null)
{
- hasQueriableIndex = true;
- if (def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN)
- hasQueriableClusteringColumnIndex = true;
+ if (operator == Relation.Type.EQ || operator == Relation.Type.IN)
- throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation", name, relation.operator()));
++ throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation", def, relation.operator()));
+ else if (!existing.isSlice())
- throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", name));
++ throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", def));
}
- restrictedColumns.add(name);
++ restrictedColumns.add(def);
+ }
- switch (def.kind)
- boolean onToken = false;
-
+ switch (relation.operator())
+ {
+ case EQ:
{
- case PARTITION_KEY:
- stmt.keyRestrictions[def.position()] = updateRestriction(cfm, def, stmt.keyRestrictions[def.position()], rel, names);
- break;
- case CLUSTERING_COLUMN:
- stmt.columnRestrictions[def.position()] = updateRestriction(cfm, def, stmt.columnRestrictions[def.position()], rel, 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 = updateRestriction(cfm, def, stmt.metadataRestrictions.get(def.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", def.name));
- stmt.metadataRestrictions.put(def.name, r);
- break;
- Term t = relation.getValue().prepare(names);
++ Term t = relation.getValue().prepare(keyspace(), defs);
+ t.collectMarkerSpecification(boundNames);
- Restriction restriction = new MultiColumnRestriction.EQ(t, onToken);
- for (CFDefinition.Name name : restrictedColumns)
- stmt.columnRestrictions[name.position] = restriction;
++ Restriction restriction = new MultiColumnRestriction.EQ(t, false);
++ for (ColumnDefinition def : restrictedColumns)
++ stmt.columnRestrictions[def.position()] = restriction;
+ break;
+ }
+ case IN:
+ {
+ Restriction restriction;
+ List<? extends Term.MultiColumnRaw> inValues = relation.getInValues();
+ if (inValues != null)
+ {
+ // we have something like "(a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) or
+ // "(a, b, c) IN (?, ?, ?)
+ List<Term> terms = new ArrayList<>(inValues.size());
+ for (Term.MultiColumnRaw tuple : inValues)
+ {
- Term t = tuple.prepare(names);
++ Term t = tuple.prepare(keyspace(), defs);
+ t.collectMarkerSpecification(boundNames);
+ terms.add(t);
+ }
+ restriction = new MultiColumnRestriction.InWithValues(terms);
+ }
+ else
+ {
+ Tuples.INRaw rawMarker = relation.getInMarker();
- AbstractMarker t = rawMarker.prepare(names);
++ AbstractMarker t = rawMarker.prepare(keyspace(), defs);
+ t.collectMarkerSpecification(boundNames);
+ restriction = new MultiColumnRestriction.InWithMarker(t);
+ }
- for (CFDefinition.Name name : restrictedColumns)
- stmt.columnRestrictions[name.position] = restriction;
++ for (ColumnDefinition def : restrictedColumns)
++ stmt.columnRestrictions[def.position()] = restriction;
+
+ break;
+ }
+ case LT:
+ case LTE:
+ case GT:
+ case GTE:
+ {
- Term t = relation.getValue().prepare(names);
++ Term t = relation.getValue().prepare(keyspace(), defs);
+ t.collectMarkerSpecification(boundNames);
- for (CFDefinition.Name name : names)
++ for (ColumnDefinition def : defs)
+ {
- Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, name);
++ Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, def);
+ if (restriction == null)
- restriction = new MultiColumnRestriction.Slice(onToken);
++ restriction = new MultiColumnRestriction.Slice(false);
+ else if (!restriction.isMultiColumn())
- throw new InvalidRequestException(String.format("Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities", name, relation));
- restriction.setBound(relation.operator(), t);
- stmt.columnRestrictions[name.position] = restriction;
++ throw new InvalidRequestException(String.format("Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities: %s", def.name, relation));
++ restriction.setBound(def.name, relation.operator(), t);
++ stmt.columnRestrictions[def.position()] = restriction;
+ }
}
}
+ }
- /*
- * At this point, the select statement if fully constructed, but we still have a few things to validate
- */
- private Restriction getExistingRestriction(SelectStatement stmt, CFDefinition.Name name)
++ private Restriction getExistingRestriction(SelectStatement stmt, ColumnDefinition def)
+ {
- switch (name.kind)
++ switch (def.kind)
+ {
- case KEY_ALIAS:
- return stmt.keyRestrictions[name.position];
- case COLUMN_ALIAS:
- return stmt.columnRestrictions[name.position];
- case VALUE_ALIAS:
- return null;
++ case PARTITION_KEY:
++ return stmt.keyRestrictions[def.position()];
++ case CLUSTERING_COLUMN:
++ return stmt.columnRestrictions[def.position()];
++ case REGULAR:
++ case STATIC:
++ return stmt.metadataRestrictions.get(def);
+ default:
- return stmt.metadataRestrictions.get(name);
++ throw new AssertionError();
+ }
+ }
- private void updateRestrictionsForRelation(SelectStatement stmt, CFDefinition.Name name, SingleColumnRelation relation, VariableSpecifications names) throws InvalidRequestException
++ private void updateRestrictionsForRelation(SelectStatement stmt, ColumnDefinition def, SingleColumnRelation relation, VariableSpecifications names) throws InvalidRequestException
+ {
- switch (name.kind)
++ switch (def.kind)
+ {
- case KEY_ALIAS:
- stmt.keyRestrictions[name.position] = updateSingleColumnRestriction(name, stmt.keyRestrictions[name.position], relation, names);
++ case PARTITION_KEY:
++ stmt.keyRestrictions[def.position()] = updateSingleColumnRestriction(def, stmt.keyRestrictions[def.position()], relation, names);
+ break;
- case COLUMN_ALIAS:
- stmt.columnRestrictions[name.position] = updateSingleColumnRestriction(name, stmt.columnRestrictions[name.position], relation, names);
++ case CLUSTERING_COLUMN:
++ stmt.columnRestrictions[def.position()] = updateSingleColumnRestriction(def, stmt.columnRestrictions[def.position()], 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 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(name, stmt.metadataRestrictions.get(name), relation, names);
++ Restriction r = updateSingleColumnRestriction(def, stmt.metadataRestrictions.get(def.name), 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));
- stmt.metadataRestrictions.put(name, r);
++ 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(CFDefinition.Name name, Restriction existingRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
++ Restriction updateSingleColumnRestriction(ColumnDefinition def, Restriction existingRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
+ {
- ColumnSpecification receiver = name;
++ ColumnSpecification receiver = def;
+ if (newRel.onToken)
+ {
- if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS)
- throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", name));
++ if (def.kind != ColumnDefinition.Kind.PARTITION_KEY)
++ throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", def.name));
+
- receiver = new ColumnSpecification(name.ksName,
- name.cfName,
++ receiver = new ColumnSpecification(def.ksName,
++ def.cfName,
+ new ColumnIdentifier("partition key token", true),
+ StorageService.getPartitioner().getTokenValidator());
+ }
+
+ switch (newRel.operator())
+ {
+ case EQ:
+ {
+ if (existingRestriction != null)
- throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", name));
- Term t = newRel.getValue().prepare(receiver);
++ throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", def.name));
++ Term t = newRel.getValue().prepare(keyspace(), receiver);
+ t.collectMarkerSpecification(boundNames);
+ existingRestriction = new SingleColumnRestriction.EQ(t, newRel.onToken);
+ }
+ break;
+ case IN:
+ if (existingRestriction != null)
- throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", name));
++ throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", def.name));
+
+ if (newRel.getInValues() == null)
+ {
+ // Means we have a "SELECT ... IN ?"
+ assert newRel.getValue() != null;
- Term t = newRel.getValue().prepare(receiver);
++ Term t = newRel.getValue().prepare(keyspace(), receiver);
+ t.collectMarkerSpecification(boundNames);
+ existingRestriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
+ }
+ else
+ {
- List<Term> inValues = new ArrayList<Term>(newRel.getInValues().size());
++ List<Term> inValues = new ArrayList<>(newRel.getInValues().size());
+ for (Term.Raw raw : newRel.getInValues())
+ {
- Term t = raw.prepare(receiver);
++ Term t = raw.prepare(keyspace(), receiver);
+ t.collectMarkerSpecification(boundNames);
+ inValues.add(t);
+ }
+ existingRestriction = new SingleColumnRestriction.InWithValues(inValues);
+ }
+ break;
+ case GT:
+ case GTE:
+ case LT:
+ case LTE:
++ {
++ 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));
++
++ Term t = newRel.getValue().prepare(keyspace(), receiver);
++ t.collectMarkerSpecification(boundNames);
++ ((SingleColumnRestriction.Slice)existingRestriction).setBound(def.name, newRel.operator(), t);
++ }
++ break;
++ case CONTAINS_KEY:
++ if (!(receiver.type instanceof MapType))
++ throw new InvalidRequestException(String.format("Cannot use CONTAINS_KEY on non-map column %s", def.name));
++ // Fallthrough on purpose
++ case CONTAINS:
+ {
++ if (!receiver.type.isCollection())
++ throw new InvalidRequestException(String.format("Cannot use %s relation on non collection column %s", newRel.operator(), def.name));
++
+ 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", 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)", name, newRel));
- Term t = newRel.getValue().prepare(receiver);
- t.collectMarkerSpecification(boundNames);
- ((SingleColumnRestriction.Slice)existingRestriction).setBound(newRel.operator(), t);
++ existingRestriction = new SingleColumnRestriction.Contains();
++ else if (!existingRestriction.isContains())
++ throw new InvalidRequestException(String.format("Collection column %s can only be restricted by CONTAINS or CONTAINS KEY", def.name));
++ boolean isKey = newRel.operator() == Relation.Type.CONTAINS_KEY;
++ receiver = makeCollectionReceiver(receiver, isKey);
++ Term t = newRel.getValue().prepare(keyspace(), receiver);
++ ((SingleColumnRestriction.Contains)existingRestriction).add(t, isKey);
+ }
- break;
+ }
+ return existingRestriction;
+ }
+
- private void processPartitionKeyRestrictions(SelectStatement stmt, CFDefinition cfDef, boolean hasQueriableIndex) throws InvalidRequestException
++ private void processPartitionKeyRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException
+ {
// If there is a queriable index, no special condition are required on the other restrictions.
// But we still need to know 2 things:
// - If we don't have a queriable index, is the query ok
@@@ -1338,7 -1734,9 +1712,9 @@@
stmt.usesSecondaryIndexing = true;
break;
}
- throw new InvalidRequestException(String.format("partition key part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cdef.name, previous));
+ throw new InvalidRequestException(String.format(
+ "Partitioning column \"%s\" cannot be restricted because the preceding column (\"%s\") is " +
- "either not restricted or is restricted by a non-EQ relation", cname, previous));
++ "either not restricted or is restricted by a non-EQ relation", cdef.name, previous));
}
else if (restriction.isOnToken())
{
@@@ -1368,27 -1766,22 +1744,22 @@@
// 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;
+ previous = cdef;
}
+ }
- // All (or none) of the partition key columns have been specified;
- // hence there is no need to turn these restrictions into index expressions.
- if (!stmt.usesSecondaryIndexing)
- stmt.restrictedColumns.removeAll(cfm.partitionKeyColumns());
-
- if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
- throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
-
- private void processColumnRestrictions(SelectStatement stmt, CFDefinition cfDef, boolean hasQueriableIndex) throws InvalidRequestException
++ private void processColumnRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException
+ {
// If a clustering key column is restricted by a non-EQ relation, all preceding
// columns must have a EQ, and all following must have no restriction. Unless
// the column is indexed that is.
- canRestrictFurtherComponents = true;
- previous = null;
+ boolean canRestrictFurtherComponents = true;
- CFDefinition.Name previous = null;
++ ColumnDefinition previous = null;
boolean previousIsSlice = false;
- iter = cfm.clusteringColumns().iterator();
- Iterator<CFDefinition.Name> iter = cfDef.clusteringColumns().iterator();
++ Iterator<ColumnDefinition> iter = cfm.clusteringColumns().iterator();
for (int i = 0; i < stmt.columnRestrictions.length; i++)
{
- CFDefinition.Name cname = iter.next();
+ ColumnDefinition cdef = iter.next();
Restriction restriction = stmt.columnRestrictions[i];
if (restriction == null)
@@@ -1410,7 -1803,8 +1781,8 @@@
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)", cdef.name, previous));
+ throw new InvalidRequestException(String.format(
- "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is either not restricted or by a non-EQ relation)", cname, previous));
++ "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is either not restricted or by a non-EQ relation)", cdef.name, previous));
}
}
else if (restriction.isSlice())
@@@ -1425,160 -1819,160 +1797,152 @@@
}
else if (restriction.isIN())
{
- // We only support IN for the last name and for compact storage so far
- // TODO: #3885 allows us to extend to non compact as well, but that remains to be done
- if (i != stmt.columnRestrictions.length - 1)
- throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cdef.name));
+ if (!restriction.isMultiColumn() && i != stmt.columnRestrictions.length - 1)
- throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cname));
- if (stmt.selectACollection())
- throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cname));
++ throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cdef.name));
+ else if (stmt.selectACollection())
- throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cdef.name));
++ throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cdef.name));
}
- previous = cname;
+ previous = cdef;
}
+ }
- // Covers indexes on the first clustering column (among others).
- if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
- stmt.usesSecondaryIndexing = true;
-
- if (!stmt.usesSecondaryIndexing)
- stmt.restrictedColumns.removeAll(cfm.clusteringColumns());
-
- // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
- // there is restrictions not covered by the PK.
- if (!stmt.metadataRestrictions.isEmpty())
- {
- if (!hasQueriableIndex)
- throw new InvalidRequestException("No indexed columns present in by-columns clause with Equal operator");
- stmt.usesSecondaryIndexing = true;
- }
+ private void validateSecondaryIndexSelections(SelectStatement stmt) throws InvalidRequestException
+ {
+ if (stmt.keyIsInRelation)
+ throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
+ // When the user only select static columns, the intent is that we don't query the whole partition but just
+ // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
+ // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
+ if (stmt.selectsOnlyStaticColumns)
+ throw new InvalidRequestException("Queries using 2ndary indexes don't support selecting only static columns");
+ }
+ private void verifyOrderingIsAllowed(SelectStatement stmt) throws InvalidRequestException
+ {
if (stmt.usesSecondaryIndexing)
- {
- if (stmt.keyIsInRelation)
- throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
- // When the user only select static columns, the intent is that we don't query the whole partition but just
- // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
- // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
- if (stmt.selectsOnlyStaticColumns)
- throw new InvalidRequestException("Queries using 2ndary indexes don't support selecting only static columns");
- }
+ throw new InvalidRequestException("ORDER BY with 2ndary indexes is not supported.");
- if (!stmt.parameters.orderings.isEmpty())
- {
- if (stmt.usesSecondaryIndexing)
- throw new InvalidRequestException("ORDER BY with 2ndary indexes is not supported.");
+ if (stmt.isKeyRange)
+ throw new InvalidRequestException("ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
+ }
- if (stmt.isKeyRange)
- throw new InvalidRequestException("ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
+ private void handleUnrecognizedOrderingColumn(ColumnIdentifier column) throws InvalidRequestException
+ {
+ if (containsAlias(column))
+ throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
+ else
+ throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
+ }
- // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting, even if we don't
- // ultimately ship them to the client (CASSANDRA-4911).
- if (stmt.keyIsInRelation)
- private void processOrderingClause(SelectStatement stmt, CFDefinition cfDef) throws InvalidRequestException
++ private void processOrderingClause(SelectStatement stmt, CFMetaData cfm) throws InvalidRequestException
+ {
+ verifyOrderingIsAllowed(stmt);
+
- // If we order an IN query, we'll have to do a manual sort post-query. Currently, this sorting requires that we
- // have queried the column on which we sort (TODO: we should update it to add the column on which we sort to the one
- // queried automatically, and then removing it from the resultSet afterwards if needed)
++ // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting, even if we don't
++ // ultimately ship them to the client (CASSANDRA-4911).
+ if (stmt.keyIsInRelation)
+ {
- stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
++ stmt.orderingIndexes = new HashMap<>();
+ for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
{
- stmt.orderingIndexes = new HashMap<>();
- for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
- {
- final ColumnDefinition def = cfm.getColumnDefinition(column);
- if (def == null)
- {
- if (containsAlias(column))
- throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
- else
- throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
- }
- final CFDefinition.Name name = cfDef.get(column);
- if (name == null)
++ final ColumnDefinition def = cfm.getColumnDefinition(column);
++ if (def == null)
+ handleUnrecognizedOrderingColumn(column);
- int index = indexOf(def, stmt.selection);
- if (index < 0)
- index = stmt.selection.addColumnForOrdering(def);
- stmt.orderingIndexes.put(def.name, index);
- }
- if (selectClause.isEmpty()) // wildcard
- {
- stmt.orderingIndexes.put(name, Iterables.indexOf(cfDef, new Predicate<CFDefinition.Name>()
- {
- public boolean apply(CFDefinition.Name n)
- {
- return name.equals(n);
- }
- }));
- }
- else
- {
- boolean hasColumn = false;
- for (int i = 0; i < selectClause.size(); i++)
- {
- RawSelector selector = selectClause.get(i);
- if (name.name.equals(selector.selectable))
- {
- stmt.orderingIndexes.put(name, i);
- hasColumn = true;
- break;
- }
- }
-
- if (!hasColumn)
- throw new InvalidRequestException("ORDER BY could not be used on columns missing in select clause.");
- }
++ int index = indexOf(def, stmt.selection);
++ if (index < 0)
++ index = stmt.selection.addColumnForOrdering(def);
++ stmt.orderingIndexes.put(def.name, index);
}
+ }
- stmt.isReversed = isReversed(stmt, cfDef);
++ stmt.isReversed = isReversed(stmt, cfm);
+ }
- Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
- int i = 0;
- for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
- {
- ColumnIdentifier column = entry.getKey();
- boolean reversed = entry.getValue();
- private boolean isReversed(SelectStatement stmt, CFDefinition cfDef) throws InvalidRequestException
++ private boolean isReversed(SelectStatement stmt, CFMetaData cfm) throws InvalidRequestException
+ {
- Boolean[] reversedMap = new Boolean[cfDef.clusteringColumnsCount()];
++ Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
+ int i = 0;
+ for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
+ {
+ ColumnIdentifier column = entry.getKey();
+ boolean reversed = entry.getValue();
- ColumnDefinition def = cfm.getColumnDefinition(column);
- if (def == null)
- {
- if (containsAlias(column))
- throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
- else
- throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
- }
- CFDefinition.Name name = cfDef.get(column);
- if (name == null)
++ ColumnDefinition def = cfm.getColumnDefinition(column);
++ if (def == null)
+ handleUnrecognizedOrderingColumn(column);
- if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
- throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
- if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
++ if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
+ throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
- if (i++ != def.position())
- throw new InvalidRequestException(String.format("Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"));
- if (i++ != name.position)
++ if (i++ != def.position())
+ throw new InvalidRequestException(String.format("Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"));
- reversedMap[def.position()] = (reversed != isReversedType(def));
- }
- reversedMap[name.position] = (reversed != isReversedType(name));
++ reversedMap[def.position()] = (reversed != isReversedType(def));
+ }
- // Check that all boolean in reversedMap, if set, agrees
- Boolean isReversed = null;
- for (Boolean b : reversedMap)
- {
- // Cell on which order is specified can be in any order
- if (b == null)
- continue;
+ // Check that all boolean in reversedMap, if set, agrees
+ Boolean isReversed = null;
+ for (Boolean b : reversedMap)
+ {
+ // Column on which order is specified can be in any order
+ if (b == null)
+ continue;
- if (isReversed == null)
- {
- isReversed = b;
- continue;
- }
- if (isReversed != b)
- throw new InvalidRequestException(String.format("Unsupported order by relation"));
+ if (isReversed == null)
+ {
+ isReversed = b;
+ continue;
}
- assert isReversed != null;
- stmt.isReversed = isReversed;
+ if (isReversed != b)
+ throw new InvalidRequestException(String.format("Unsupported order by relation"));
}
+ assert isReversed != null;
+ return isReversed;
+ }
- // Make sure this queries is allowed (note: non key range non indexed cannot involve filtering underneath)
+ /** If ALLOW FILTERING was not specified, this verifies that it is not needed */
+ private void checkNeedsFiltering(SelectStatement stmt) throws InvalidRequestException
+ {
+ // non-key-range non-indexed queries cannot involve filtering underneath
if (!parameters.allowFiltering && (stmt.isKeyRange || stmt.usesSecondaryIndexing))
{
// We will potentially filter data if either:
// - Have more than one IndexExpression
// - Have no index expression and the column filter is not the identity
- if (stmt.restrictedNames.size() > 1 || (stmt.restrictedNames.isEmpty() && !stmt.columnFilterIsIdentity()))
+ if (stmt.restrictedColumns.size() > 1 || (stmt.restrictedColumns.isEmpty() && !stmt.columnFilterIsIdentity()))
- throw new InvalidRequestException("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");
+ throw new InvalidRequestException("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");
}
-
- return new ParsedStatement.Prepared(stmt, names);
}
- private void validateDistinctSelection(Collection<CFDefinition.Name> requestedColumns, Collection<CFDefinition.Name> partitionKey)
+ private int indexOf(ColumnDefinition def, Selection selection)
+ {
+ return indexOf(def, selection.getColumns().iterator());
+ }
+
+ private int indexOf(final ColumnDefinition def, Iterator<ColumnDefinition> defs)
+ {
+ return Iterators.indexOf(defs, new Predicate<ColumnDefinition>()
+ {
+ public boolean apply(ColumnDefinition n)
+ {
+ return def.name.equals(n.name);
+ }
+ });
+ }
+
+ private void validateDistinctSelection(Collection<ColumnDefinition> requestedColumns, Collection<ColumnDefinition> partitionKey)
throws InvalidRequestException
{
- for (CFDefinition.Name name : requestedColumns)
- if (!partitionKey.contains(name))
- throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", name));
+ for (ColumnDefinition def : requestedColumns)
+ if (!partitionKey.contains(def))
+ throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", def.name));
- for (CFDefinition.Name name : partitionKey)
- if (!requestedColumns.contains(name))
- throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
+ for (ColumnDefinition def : partitionKey)
+ if (!requestedColumns.contains(def))
+ throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name));
}
private boolean containsAlias(final ColumnIdentifier name)
@@@ -1597,114 -1991,6 +1961,23 @@@
return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
}
- Restriction updateRestriction(CFMetaData cfm, ColumnDefinition def, Restriction restriction, Relation newRel, VariableSpecifications boundNames) throws InvalidRequestException
- {
- ColumnSpecification receiver = def;
- if (newRel.onToken)
- {
- if (def.kind != ColumnDefinition.Kind.PARTITION_KEY)
- throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", def.name));
-
- receiver = new ColumnSpecification(def.ksName,
- def.cfName,
- new ColumnIdentifier("partition key token", true),
- StorageService.getPartitioner().getTokenValidator());
- }
-
- // We can only use the tuple notation of #4851 on clustering columns for now
- if (newRel.previousInTuple != null && def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
- throw new InvalidRequestException(String.format("Tuple notation can only be used on clustering columns but found on %s", def.name));
-
- switch (newRel.operator())
- {
- case EQ:
- {
- if (restriction != null)
- throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", def.name));
- Term t = newRel.getValue().prepare(keyspace(), receiver);
- t.collectMarkerSpecification(boundNames);
- restriction = new Restriction.EQ(t, newRel.onToken);
- }
- break;
- case IN:
- if (restriction != null)
- throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", def.name));
-
- if (newRel.getInValues() == null)
- {
- // Means we have a "SELECT ... IN ?"
- assert newRel.getValue() != null;
- Term t = newRel.getValue().prepare(keyspace(), receiver);
- t.collectMarkerSpecification(boundNames);
- restriction = Restriction.IN.create(t);
- }
- else
- {
- List<Term> inValues = new ArrayList<Term>(newRel.getInValues().size());
- for (Term.Raw raw : newRel.getInValues())
- {
- Term t = raw.prepare(keyspace(), receiver);
- t.collectMarkerSpecification(boundNames);
- inValues.add(t);
- }
- restriction = Restriction.IN.create(inValues);
- }
- break;
- case GT:
- case GTE:
- case LT:
- case LTE:
- {
- if (restriction == null)
- restriction = new Restriction.Slice(newRel.onToken);
- else if (!restriction.isSlice())
- throw new InvalidRequestException(String.format("%s cannot be restricted by both an equal and an inequal relation", def.name));
- Term t = newRel.getValue().prepare(keyspace(), receiver);
- t.collectMarkerSpecification(boundNames);
- if (newRel.previousInTuple != null && (def.position() == 0 || !cfm.clusteringColumns().get(def.position() - 1).name.equals(newRel.previousInTuple)))
- throw new InvalidRequestException(String.format("Invalid tuple notation, column %s is not before column %s in the clustering order", newRel.previousInTuple, def.name));
- ((Restriction.Slice)restriction).setBound(def.name, newRel.operator(), t, newRel.previousInTuple);
- }
- break;
- case CONTAINS_KEY:
- if (!(receiver.type instanceof MapType))
- throw new InvalidRequestException(String.format("Cannot use CONTAINS_KEY on non-map column %s", def.name));
- // Fallthrough on purpose
- case CONTAINS:
- {
- if (!receiver.type.isCollection())
- throw new InvalidRequestException(String.format("Cannot use %s relation on non collection column %s", newRel.operator(), def.name));
-
- if (restriction == null)
- restriction = new Restriction.Contains();
- else if (!restriction.isContains())
- throw new InvalidRequestException(String.format("Collection column %s can only be restricted by CONTAINS or CONTAINS KEY", def.name));
- boolean isKey = newRel.operator() == Relation.Type.CONTAINS_KEY;
- receiver = makeCollectionReceiver(receiver, isKey);
- Term t = newRel.getValue().prepare(keyspace(), receiver);
- ((Restriction.Contains)restriction).add(t, isKey);
- }
- }
- return restriction;
- }
-
+ private static ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+ {
+ assert collection.type.isCollection();
+ switch (((CollectionType)collection.type).kind)
+ {
+ case LIST:
+ assert !isKey;
+ return Lists.valueSpecOf(collection);
+ case SET:
+ assert !isKey;
+ return Sets.valueSpecOf(collection);
+ case MAP:
+ return isKey ? Maps.keySpecOf(collection) : Maps.valueSpecOf(collection);
+ }
+ throw new AssertionError();
+ }
+
@Override
public String toString()
{