Merge branch 'cassandra-2.0' into cassandra-2.1
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0452e74f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0452e74f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0452e74f Branch: refs/heads/cassandra-2.2 Commit: 0452e74f59182aee515cc6920e35287def86c9fe Parents: 9966419 f32cff8 Author: Sam Tunnicliffe <s...@beobal.com> Authored: Thu Jun 18 17:32:04 2015 +0100 Committer: Sam Tunnicliffe <s...@beobal.com> Committed: Thu Jun 18 17:34:27 2015 +0100 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../cassandra/cql3/ColumnSpecification.java | 22 ++ .../cql3/statements/SelectStatement.java | 44 +++- .../cassandra/cql3/statements/Selection.java | 79 +++--- .../cql3/statements/SelectionColumnMapping.java | 107 ++++++++ .../cql3/statements/SelectionColumns.java | 18 ++ .../statements/SelectionColumnMappingTest.java | 252 +++++++++++++++++++ 7 files changed, 492 insertions(+), 31 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0452e74f/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index 1d72c9a,a235528..899ea7c --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,6 -1,5 +1,7 @@@ -2.0.16: +2.1.7 + * Fix memory leak in Ref due to ConcurrentLinkedQueue.remove() behaviour (CASSANDRA-9549) +Merged from 2.0 + * Expose some internals of SelectStatement for inspection (CASSANDRA-9532) * ArrivalWindow should use primitives (CASSANDRA-9496) * Periodically submit background compaction tasks (CASSANDRA-9592) * Set HAS_MORE_PAGES flag to false when PagingState is null (CASSANDRA-9571) http://git-wip-us.apache.org/repos/asf/cassandra/blob/0452e74f/src/java/org/apache/cassandra/cql3/ColumnSpecification.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/ColumnSpecification.java index d2e08f9,089a1c5..f5f921d --- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java +++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java @@@ -33,4 -35,31 +35,24 @@@ public class ColumnSpecificatio this.name = name; this.type = type; } + - @Override - public String toString() - { - // Not fully conventional, but convenient (for error message to users in particular) - return name.toString(); - } - + public boolean equals(Object obj) + { + if (null == obj) + return false; + + if(!(obj instanceof ColumnSpecification)) + return false; + + ColumnSpecification other = (ColumnSpecification)obj; + return Objects.equal(ksName, other.ksName) + && Objects.equal(cfName, other.cfName) + && Objects.equal(name, other.name) + && Objects.equal(type, other.type); + } + + public int hashCode() + { + return Objects.hashCode(ksName, cfName, name, type); + } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/0452e74f/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 51f4941,1c19760..d0566eb --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@@ -58,9 -60,12 +58,13 @@@ import org.slf4j.LoggerFactory /** * Encapsulates a completely parsed SELECT query, including the target * column family, expression, result count, and ordering clause. + * + * A number of public methods here are only used internally. However, + * many of these are made accessible for the benefit of custom + * QueryHandler implementations, so before reducing their accessibility + * due consideration should be given. */ -public class SelectStatement implements CQLStatement, MeasurableForPreparedCache +public class SelectStatement implements CQLStatement { private static final Logger logger = LoggerFactory.getLogger(SelectStatement.class); @@@ -170,6 -168,20 +174,9 @@@ : selection.getResultMetadata(); } - public long measureForPreparedCache(MemoryMeter meter) - { - return meter.measure(this) - + meter.measureDeep(parameters) - + meter.measureDeep(selection) - + (limit == null ? 0 : meter.measureDeep(limit)) - + meter.measureDeep(keyRestrictions) - + meter.measureDeep(columnRestrictions) - + meter.measureDeep(metadataRestrictions) - + meter.measureDeep(restrictedNames) - + (sliceRestriction == null ? 0 : meter.measureDeep(sliceRestriction)) - + (orderingIndexes == null ? 0 : meter.measureDeep(orderingIndexes)); - } - ++ /** ++ * May be used by custom QueryHandler implementations ++ */ public int getBoundTerms() { return boundTerms; @@@ -367,9 -403,9 +382,12 @@@ : new RangeSliceCommand(keyspace(), columnFamily(), now, filter, keyBounds, expressions, limit, !parameters.isDistinct, false); } - private AbstractBounds<RowPosition> getKeyBounds(QueryOptions options) throws InvalidRequestException - private AbstractBounds<RowPosition> getKeyBounds(CFDefinition cfDef, List<ByteBuffer> variables) throws InvalidRequestException ++ /** ++ * May be used by custom QueryHandler implementations ++ */ ++ public AbstractBounds<RowPosition> getKeyBounds(QueryOptions options) throws InvalidRequestException { - IPartitioner<?> p = StorageService.getPartitioner(); + IPartitioner p = StorageService.getPartitioner(); if (onToken) { @@@ -550,7 -591,10 +568,10 @@@ return new SliceQueryFilter(slices, isReversed, limit, toGroup); } - private int getLimit(QueryOptions options) throws InvalidRequestException + /** + * May be used by custom QueryHandler implementations + */ - public int getLimit(List<ByteBuffer> variables) throws InvalidRequestException ++ public int getLimit(QueryOptions options) throws InvalidRequestException { int l = Integer.MAX_VALUE; if (limit != null) @@@ -1024,21 -1064,29 +1045,24 @@@ { Restriction.Slice slice = (Restriction.Slice)r; assert slice.hasBound(b); - return slice.bound(b, variables); + ByteBuffer val = slice.bound(b, options); + if (val == null) + throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r)); + return val; } - private List<ByteBuffer> getRequestedBound(CFDefinition cfDef, - Bound b, - List<ByteBuffer> variables) throws InvalidRequestException + private List<Composite> getRequestedBound(Bound b, QueryOptions options) throws InvalidRequestException { - assert isColumnRange(cfDef); - return buildBound(b, - new ArrayList<Name>(cfDef.clusteringColumns()), - columnRestrictions, - isReversed, - cfDef, - cfDef.getColumnNameBuilder(), - variables); + assert isColumnRange(); + return buildBound(b, cfm.clusteringColumns(), columnRestrictions, isReversed, cfm.comparator, options); } + /** + * May be used by custom QueryHandler implementations + */ - public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException + public List<IndexExpression> getValidatedIndexExpressions(QueryOptions options) throws InvalidRequestException { - if (!usesSecondaryIndexing || restrictedNames.isEmpty()) + if (!usesSecondaryIndexing || restrictedColumns.isEmpty()) return Collections.emptyList(); List<IndexExpression> expressions = new ArrayList<IndexExpression>(); @@@ -1364,21 -1463,38 +1388,35 @@@ return true; } - private boolean hasClusteringColumnsRestriction() + /** + * May be used by custom QueryHandler implementations + */ - public boolean hasPartitionKeyRestriction() ++ public boolean hasClusteringColumnsRestriction() { - for (int i = 0; i < keyRestrictions.length; i++) - if (keyRestrictions[i] != null) + for (int i = 0; i < columnRestrictions.length; i++) + if (columnRestrictions[i] != null) return true; return false; } + /** + * May be used by custom QueryHandler implementations + */ - public boolean hasClusteringColumnsRestriction() ++ public boolean hasPartitionKeyRestriction() + { - for (int i = 0; i < columnRestrictions.length; i++) - if (columnRestrictions[i] != null) ++ for (int i = 0; i < keyRestrictions.length; i++) ++ if (keyRestrictions[i] != null) + return true; + return false; + } + - private void validateDistinctSelection(CFDefinition cfDef) + private void validateDistinctSelection() throws InvalidRequestException { - Collection<CFDefinition.Name> requestedColumns = selection.getColumns(); - for (CFDefinition.Name name : requestedColumns) - { - if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS && name.kind != CFDefinition.Name.Kind.STATIC) - throw new InvalidRequestException(String.format( - "SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", name)); - } + Collection<ColumnDefinition> requestedColumns = selection.getColumns(); + for (ColumnDefinition def : requestedColumns) + if (def.kind != ColumnDefinition.Kind.PARTITION_KEY && def.kind != ColumnDefinition.Kind.STATIC) + throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", def.name)); // If it's a key range, we require that all partition key columns are selected so we don't have to bother with post-query grouping. if (!isKeyRange) http://git-wip-us.apache.org/repos/asf/cassandra/blob/0452e74f/src/java/org/apache/cassandra/cql3/statements/Selection.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/statements/Selection.java index ff808bb,50a34bf..83cbfe8 --- a/src/java/org/apache/cassandra/cql3/statements/Selection.java +++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java @@@ -18,10 -18,9 +18,9 @@@ package org.apache.cassandra.cql3.statements; import java.nio.ByteBuffer; - import java.util.ArrayList; - import java.util.List; + import java.util.*; -import com.google.common.collect.*; +import com.google.common.collect.Iterators; import org.apache.cassandra.cql3.*; import org.apache.cassandra.cql3.functions.Function; @@@ -42,15 -37,15 +41,17 @@@ import org.apache.cassandra.utils.ByteB public abstract class Selection { - private final List<CFDefinition.Name> columns; - private final SelectionColumns columnMapping; + private final List<ColumnDefinition> columns; ++ private final SelectionColumnMapping columnMapping; + private final ResultSet.Metadata metadata; private final boolean collectTimestamps; private final boolean collectTTLs; - protected Selection(List<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs) - protected Selection(List<CFDefinition.Name> columns, SelectionColumns columnMapping, boolean collectTimestamps, boolean collectTTLs) ++ protected Selection(List<ColumnDefinition> columns, SelectionColumnMapping columnMapping, boolean collectTimestamps, boolean collectTTLs) { this.columns = columns; - this.metadata = new ResultSet.Metadata(metadata); + this.columnMapping = columnMapping; ++ this.metadata = new ResultSet.Metadata(columnMapping.getColumnSpecifications()); this.collectTimestamps = collectTimestamps; this.collectTTLs = collectTTLs; } @@@ -106,74 -95,60 +107,80 @@@ return idx; } - private static Selector makeSelector(CFMetaData cfm, RawSelector raw, List<ColumnDefinition> defs, List<ColumnSpecification> metadata) throws InvalidRequestException - private static Selector makeSelector(CFDefinition cfDef, - RawSelector raw, - List<CFDefinition.Name> names, - SelectionColumnMapping columnMapping) throws InvalidRequestException ++ private static Selector makeSelector(CFMetaData cfm, RawSelector raw, List<ColumnDefinition> defs, SelectionColumnMapping columnMapping) throws InvalidRequestException { - Selectable selectable = raw.selectable.prepare(cfDef.cfm); - return makeSelector(cfDef, selectable, raw.alias, names, columnMapping); + Selectable selectable = raw.selectable.prepare(cfm); - return makeSelector(cfm, selectable, raw.alias, defs, metadata); ++ return makeSelector(cfm, selectable, raw.alias, defs, columnMapping); } - private static Selector makeSelector(CFMetaData cfm, Selectable selectable, ColumnIdentifier alias, List<ColumnDefinition> defs, List<ColumnSpecification> metadata) throws InvalidRequestException - private static Selector makeSelector(CFDefinition cfDef, - Selectable selectable, - ColumnIdentifier alias, - List<CFDefinition.Name> names, - SelectionColumnMapping columnMapping) throws InvalidRequestException ++ private static Selector makeSelector(CFMetaData cfm, Selectable selectable, ColumnIdentifier alias, List<ColumnDefinition> defs, SelectionColumnMapping columnMapping) throws InvalidRequestException { if (selectable instanceof ColumnIdentifier) { - CFDefinition.Name name = cfDef.get((ColumnIdentifier) selectable); - if (name == null) + ColumnDefinition def = cfm.getColumnDefinition((ColumnIdentifier)selectable); + if (def == null) throw new InvalidRequestException(String.format("Undefined name %s in selection clause", selectable)); + - if (metadata != null) - metadata.add(alias == null ? def : makeAliasSpec(cfm, def.type, alias)); + if (columnMapping != null) - columnMapping.addMapping(alias == null ? name : makeAliasSpec(cfDef, name.type, alias), name); - return new SimpleSelector(name.toString(), addAndGetIndex(name, names), name.type); ++ columnMapping.addMapping(alias == null ? def : makeAliasSpec(cfm, def.type, alias), def); + return new SimpleSelector(def.name.toString(), addAndGetIndex(def, defs), def.type); } else if (selectable instanceof Selectable.WritetimeOrTTL) { Selectable.WritetimeOrTTL tot = (Selectable.WritetimeOrTTL)selectable; - CFDefinition.Name name = cfDef.get(tot.id); - if (name == null) + ColumnDefinition def = cfm.getColumnDefinition(tot.id); + if (def == null) throw new InvalidRequestException(String.format("Undefined name %s in selection clause", tot.id)); - if (name.isPrimaryKeyColumn()) - throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", name)); - if (name.type.isCollection()) + if (def.isPrimaryKeyColumn()) + throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", def.name)); + if (def.type.isCollection()) throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl")); + - if (metadata != null) - metadata.add(makeWritetimeOrTTLSpec(cfm, tot, alias)); + if (columnMapping != null) - columnMapping.addMapping(makeWritetimeOrTTLSpec(cfDef, tot, alias), name); - return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime); ++ columnMapping.addMapping(makeWritetimeOrTTLSpec(cfm, tot, alias), def); + return new WritetimeOrTTLSelector(def.name.toString(), addAndGetIndex(def, defs), tot.isWritetime); + } + else if (selectable instanceof Selectable.WithFieldSelection) + { + Selectable.WithFieldSelection withField = (Selectable.WithFieldSelection)selectable; - Selector selected = makeSelector(cfm, withField.selected, null, defs, null); ++ // use a temporary columns mapping to collect the underlying column from the type selectable ++ SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping(); ++ Selector selected = makeSelector(cfm, withField.selected, null, defs, tmpMapping); + AbstractType<?> type = selected.getType(); + if (!(type instanceof UserType)) + throw new InvalidRequestException(String.format("Invalid field selection: %s of type %s is not a user type", withField.selected, type.asCQL3Type())); + + UserType ut = (UserType)type; + for (int i = 0; i < ut.size(); i++) + { + if (!ut.fieldName(i).equals(withField.field.bytes)) + continue; + - if (metadata != null) - metadata.add(makeFieldSelectSpec(cfm, withField, ut.fieldType(i), alias)); ++ if (columnMapping != null) ++ columnMapping.addMapping(makeFieldSelectSpec(cfm, withField, ut.fieldType(i), alias), ++ tmpMapping.getMappings().values()); + return new FieldSelector(ut, i, selected); + } + throw new InvalidRequestException(String.format("%s of type %s has no field %s", withField.selected, type.asCQL3Type(), withField.field)); } else { Selectable.WithFunction withFun = (Selectable.WithFunction)selectable; List<Selector> args = new ArrayList<Selector>(withFun.args.size()); - // use a temporary column mapping to collate the columns used by all the function args ++ // use a temporary columns mapping to collate the columns used by all the function args + SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping(); - for (Selectable rawArg : withFun.args) - args.add(makeSelector(cfDef, rawArg, null, names, tmpMapping)); + for (Selectable arg : withFun.args) - args.add(makeSelector(cfm, arg, null, defs, null)); ++ args.add(makeSelector(cfm, arg, null, defs, tmpMapping)); - AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfDef.cfm.ksName, cfDef.cfm.cfName); + AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfm.ksName, cfm.cfName); if (returnType == null) throw new InvalidRequestException(String.format("Unknown function '%s'", withFun.functionName)); - ColumnSpecification spec = makeFunctionSpec(cfDef, withFun, returnType, alias); - Function fun = Functions.get(withFun.functionName, args, spec); + + ColumnSpecification spec = makeFunctionSpec(cfm, withFun, returnType, alias); + Function fun = Functions.get(cfm.ksName, withFun.functionName, args, spec); - if (metadata != null) - metadata.add(spec); + if (columnMapping != null) + columnMapping.addMapping(spec, tmpMapping.getMappings().values()); ++ return new FunctionSelector(fun, args); } } @@@ -208,44 -175,49 +215,48 @@@ returnType); } - private static ColumnSpecification makeAliasSpec(CFDefinition cfDef, AbstractType<?> type, ColumnIdentifier alias) + private static ColumnSpecification makeAliasSpec(CFMetaData cfm, AbstractType<?> type, ColumnIdentifier alias) { - return new ColumnSpecification(cfDef.cfm.ksName, cfDef.cfm.cfName, alias, type); + return new ColumnSpecification(cfm.ksName, cfm.cfName, alias, type); } - public static Selection fromSelectors(CFDefinition cfDef, List<RawSelector> rawSelectors) throws InvalidRequestException + public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors) throws InvalidRequestException { - boolean needsProcessing = selectionsNeedProcessing(rawSelectors); - - if (needsProcessing) + if (requiresProcessing(rawSelectors)) { - List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(); + List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>(); - List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size()); + SelectionColumnMapping columnMapping = SelectionColumnMapping.newMapping(); List<Selector> selectors = new ArrayList<Selector>(rawSelectors.size()); boolean collectTimestamps = false; boolean collectTTLs = false; for (RawSelector rawSelector : rawSelectors) { - Selector selector = makeSelector(cfm, rawSelector, defs, metadata); - Selector selector = makeSelector(cfDef, rawSelector, names, columnMapping); ++ Selector selector = makeSelector(cfm, rawSelector, defs, columnMapping); selectors.add(selector); collectTimestamps |= selector.usesTimestamps(); collectTTLs |= selector.usesTTLs(); } - return new SelectionWithProcessing(defs, metadata, selectors, collectTimestamps, collectTTLs); - return new SelectionWithProcessing(names, columnMapping, selectors, collectTimestamps, collectTTLs); ++ return new SelectionWithProcessing(defs, columnMapping, selectors, collectTimestamps, collectTTLs); } else { - List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(rawSelectors.size()); + List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>(rawSelectors.size()); - List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size()); + SelectionColumnMapping columnMapping = SelectionColumnMapping.newMapping(); for (RawSelector rawSelector : rawSelectors) { assert rawSelector.selectable instanceof ColumnIdentifier.Raw; - ColumnIdentifier id = ((ColumnIdentifier.Raw)rawSelector.selectable).prepare(cfDef.cfm); - CFDefinition.Name name = cfDef.get(id); - if (name == null) + ColumnIdentifier id = (ColumnIdentifier) rawSelector.selectable.prepare(cfm); + ColumnDefinition def = cfm.getColumnDefinition(id); + if (def == null) throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id)); - names.add(name); - columnMapping.addMapping(rawSelector.alias == null ? name : makeAliasSpec(cfDef, - name.type, - rawSelector.alias), - name); ++ + defs.add(def); - metadata.add(rawSelector.alias == null ? def : makeAliasSpec(cfm, def.type, rawSelector.alias)); ++ columnMapping.addMapping(rawSelector.alias == null ? def : makeAliasSpec(cfm, ++ def.type, ++ rawSelector.alias), ++ def); } - return new SimpleSelection(defs, metadata, false); - return new SimpleSelection(names, columnMapping, false); ++ return new SimpleSelection(defs, columnMapping, false); } } @@@ -345,12 -339,12 +364,12 @@@ { private final boolean isWildcard; - public SimpleSelection(List<CFDefinition.Name> columns, boolean isWildcard) + public SimpleSelection(List<ColumnDefinition> columns, boolean isWildcard) { - this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard); + this(columns, SelectionColumnMapping.simpleMapping(columns), isWildcard); } - public SimpleSelection(List<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean isWildcard) - public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping, boolean isWildcard) ++ public SimpleSelection(List<ColumnDefinition> columns, SelectionColumnMapping columnMapping, boolean isWildcard) { /* * In theory, even a simple selection could have multiple time the same column, so we @@@ -414,34 -417,19 +433,38 @@@ return rs.current.get(idx); } - public boolean isAssignableTo(ColumnSpecification receiver) + public AbstractType<?> getType() { - return receiver.type.isValueCompatibleWith(type); + return type; } - public boolean usesTimestamps() + @Override + public String toString() { - return false; + return columnName; } + } - public boolean usesTTLs() + private static class SelectionWithProcessing extends Selection + { + private final List<Selector> selectors; + - public SelectionWithProcessing(List<ColumnDefinition> columns, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs) ++ public SelectionWithProcessing(List<ColumnDefinition> columns, ++ SelectionColumnMapping columnMapping, ++ List<Selector> selectors, ++ boolean collectTimestamps, ++ boolean collectTTLs) { - super(columns, metadata, collectTimestamps, collectTTLs); - return false; ++ super(columns, columnMapping, collectTimestamps, collectTTLs); + this.selectors = selectors; + } + + protected List<ByteBuffer> handleRow(ResultSetBuilder rs) throws InvalidRequestException + { + List<ByteBuffer> result = new ArrayList<>(); + for (Selector selector : selectors) + result.add(selector.compute(rs)); + return result; } @Override http://git-wip-us.apache.org/repos/asf/cassandra/blob/0452e74f/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java index 0000000,d09612f..4a6955f mode 000000,100644..100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java @@@ -1,0 -1,106 +1,107 @@@ + package org.apache.cassandra.cql3.statements; + + import java.util.*; + + import com.google.common.base.Function; + import com.google.common.base.Joiner; ++import com.google.common.base.Objects; + import com.google.common.collect.*; + -import org.apache.cassandra.cql3.CFDefinition; ++import org.apache.cassandra.config.ColumnDefinition; + import org.apache.cassandra.cql3.ColumnSpecification; + + public class SelectionColumnMapping implements SelectionColumns + { - + // Uses LinkedHashMultimap because ordering of keys must be maintained - private final LinkedHashMultimap<ColumnSpecification, CFDefinition.Name> columnMappings; ++ private final LinkedHashMultimap<ColumnSpecification, ColumnDefinition> columnMappings; + + private SelectionColumnMapping() + { + this.columnMappings = LinkedHashMultimap.create(); + } + + protected static SelectionColumnMapping newMapping() + { + return new SelectionColumnMapping(); + } + - protected static SelectionColumnMapping simpleMapping(List<CFDefinition.Name> columnDefinitions) ++ protected static SelectionColumnMapping simpleMapping(List<ColumnDefinition> columnDefinitions) + { + SelectionColumnMapping mapping = new SelectionColumnMapping(); - for (CFDefinition.Name def: columnDefinitions) ++ for (ColumnDefinition def: columnDefinitions) + mapping.addMapping(def, def); + return mapping; + } + - protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, CFDefinition.Name column) ++ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, ColumnDefinition column) + { + columnMappings.put(colSpec, column); + return this; + } + - protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<CFDefinition.Name> columns) ++ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<ColumnDefinition> columns) + { + columnMappings.putAll(colSpec, columns); + return this; + } + + public List<ColumnSpecification> getColumnSpecifications() + { + // return a mutable copy as we may add extra columns + // for ordering (CASSANDRA-4911 & CASSANDRA-8286) - return new ArrayList(columnMappings.keySet()); ++ return Lists.newArrayList(columnMappings.keySet()); + } + - public Multimap<ColumnSpecification, CFDefinition.Name> getMappings() ++ public Multimap<ColumnSpecification, ColumnDefinition> getMappings() + { + return Multimaps.unmodifiableMultimap(columnMappings); + } + + public boolean equals(Object obj) + { + if (obj == null) + return false; + - if (!(obj instanceof SelectionColumns)) ++ if (!(obj instanceof SelectionColumnMapping)) + return false; + - return Objects.equals(columnMappings, ((SelectionColumns) obj).getMappings()); ++ return Objects.equal(this.columnMappings, ((SelectionColumnMapping) obj).columnMappings); + } + + public int hashCode() + { + return Objects.hashCode(columnMappings); + } + + public String toString() + { - final Function<CFDefinition.Name, String> getDefName = new Function<CFDefinition.Name, String>() ++ final Function<ColumnDefinition, String> getDefName = new Function<ColumnDefinition, String>() + { - public String apply(CFDefinition.Name name) ++ public String apply(ColumnDefinition columnDefinition) + { - return name.toString(); ++ return columnDefinition.name.toString(); + } + }; - Function<Map.Entry<ColumnSpecification, Collection<CFDefinition.Name>>, String> mappingEntryToString = - new Function<Map.Entry<ColumnSpecification, Collection<CFDefinition.Name>>, String>(){ - public String apply(Map.Entry<ColumnSpecification, Collection<CFDefinition.Name>> entry) ++ Function<Map.Entry<ColumnSpecification, Collection<ColumnDefinition>>, String> mappingEntryToString = ++ new Function<Map.Entry<ColumnSpecification, Collection<ColumnDefinition>>, String>(){ ++ public String apply(Map.Entry<ColumnSpecification, Collection<ColumnDefinition>> entry) + { + StringBuilder builder = new StringBuilder(); + builder.append(entry.getKey().name.toString()); + builder.append(":["); + builder.append(Joiner.on(',').join(Iterables.transform(entry.getValue(), getDefName))); + builder.append("]"); + return builder.toString(); + } + }; + + StringBuilder builder = new StringBuilder(); + builder.append("{ "); + builder.append(Joiner.on(", ") + .join(Iterables.transform(columnMappings.asMap().entrySet(), + mappingEntryToString))); + builder.append(" }"); + return builder.toString(); + } ++ + } http://git-wip-us.apache.org/repos/asf/cassandra/blob/0452e74f/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java index 0000000,3053f99..5b18eff mode 000000,100644..100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java @@@ -1,0 -1,19 +1,18 @@@ + package org.apache.cassandra.cql3.statements; + + import java.util.List; + + import com.google.common.collect.Multimap; + -import org.apache.cassandra.cql3.CFDefinition; ++import org.apache.cassandra.config.ColumnDefinition; + import org.apache.cassandra.cql3.ColumnSpecification; + + /** + * Represents a mapping between the actual columns used to satisfy a Selection + * and the column definitions included in the resultset metadata for the query. + */ + public interface SelectionColumns + { + List<ColumnSpecification> getColumnSpecifications(); - Multimap<ColumnSpecification, CFDefinition.Name> getMappings(); ++ Multimap<ColumnSpecification, ColumnDefinition> getMappings(); + } - http://git-wip-us.apache.org/repos/asf/cassandra/blob/0452e74f/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java ---------------------------------------------------------------------- diff --cc test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java index 0000000,9c31653..09b2bdd mode 000000,100644..100644 --- a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java @@@ -1,0 -1,244 +1,252 @@@ + package org.apache.cassandra.cql3.statements; + + import java.util.ArrayList; + import java.util.List; + -import org.junit.BeforeClass; + import org.junit.Test; + -import org.apache.cassandra.SchemaLoader; ++import org.apache.cassandra.config.ColumnDefinition; + import org.apache.cassandra.config.Schema; + import org.apache.cassandra.cql3.*; -import org.apache.cassandra.db.ConsistencyLevel; + import org.apache.cassandra.db.marshal.*; + import org.apache.cassandra.exceptions.RequestValidationException; + import org.apache.cassandra.service.ClientState; + -import static org.apache.cassandra.cql3.QueryProcessor.process; + import static org.junit.Assert.assertEquals; + import static org.junit.Assert.assertTrue; + -public class SelectionColumnMappingTest ++public class SelectionColumnMappingTest extends CQLTester + { - static String KEYSPACE = "selection_column_mapping_test_ks"; - String tableName = "test_table"; - - @BeforeClass - public static void setupSchema() throws Throwable - { - SchemaLoader.loadSchema(); - executeSchemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s " + - "WITH replication = {'class': 'SimpleStrategy', " + - " 'replication_factor': '1'}", - KEYSPACE)); - } ++ String tableName; ++ String typeName; + + @Test + public void testSelectionColumnMapping() throws Throwable + { + // Organised as a single test to avoid the overhead of + // table creation for each variant - tableName = "table1"; - createTable("CREATE TABLE %s (" + - " k int PRIMARY KEY," + - " v1 int," + - " v2 ascii)"); ++ ++ typeName = createType("CREATE TYPE %s (f1 int, f2 text)"); ++ tableName = createTable("CREATE TABLE %s (" + ++ " k int PRIMARY KEY," + ++ " v1 int," + ++ " v2 ascii," + ++ " v3 frozen<" + typeName + ">)"); + testSimpleTypes(); + testWildcard(); + testSimpleTypesWithAliases(); ++ testUserTypes(); ++ testUserTypesWithAliases(); + testWritetimeAndTTL(); + testWritetimeAndTTLWithAliases(); + testFunction(); + testFunctionWithAlias(); + testMultipleAliasesOnSameColumn(); + testMixedColumnTypes(); + } + + @Test + public void testMultipleArgumentFunction() throws Throwable + { + // token() is currently the only function which accepts multiple arguments - tableName = "table2"; - createTable("CREATE TABLE %s (a int, b text, PRIMARY KEY ((a, b)))"); ++ tableName = createTable("CREATE TABLE %s (a int, b text, PRIMARY KEY ((a, b)))"); + ColumnSpecification tokenSpec = columnSpecification("token(a, b)", BytesType.instance); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(tokenSpec, columnDefinitions("a", "b")); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(tokenSpec, columnDefinitions("a", "b")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT token(a,b) FROM %s")); + } + + private void testSimpleTypes() throws Throwable + { + // simple column identifiers without aliases are represented in + // ResultSet.Metadata by the underlying ColumnDefinition - CFDefinition.Name kDef = columnDefinition("k"); - CFDefinition.Name v1Def = columnDefinition("v1"); - CFDefinition.Name v2Def = columnDefinition("v2"); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(kDef, columnDefinition("k")) - .addMapping(v1Def, columnDefinition("v1")) - .addMapping(v2Def, columnDefinition("v2")); ++ ColumnDefinition kDef = columnDefinition("k"); ++ ColumnDefinition v1Def = columnDefinition("v1"); ++ ColumnDefinition v2Def = columnDefinition("v2"); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(kDef, columnDefinition("k")) ++ .addMapping(v1Def, columnDefinition("v1")) ++ .addMapping(v2Def, columnDefinition("v2")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT k, v1, v2 FROM %s")); + } + + private void testWildcard() throws Throwable + { + // Wildcard select should behave just as though we had + // explicitly selected each column - CFDefinition.Name kDef = columnDefinition("k"); - CFDefinition.Name v1Def = columnDefinition("v1"); - CFDefinition.Name v2Def = columnDefinition("v2"); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(kDef, columnDefinition("k")) - .addMapping(v1Def, columnDefinition("v1")) - .addMapping(v2Def, columnDefinition("v2")); ++ ColumnDefinition kDef = columnDefinition("k"); ++ ColumnDefinition v1Def = columnDefinition("v1"); ++ ColumnDefinition v2Def = columnDefinition("v2"); ++ ColumnDefinition v3Def = columnDefinition("v3"); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(kDef, columnDefinition("k")) ++ .addMapping(v1Def, columnDefinition("v1")) ++ .addMapping(v2Def, columnDefinition("v2")) ++ .addMapping(v3Def, columnDefinition("v3")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT * FROM %s")); + } + + private void testSimpleTypesWithAliases() throws Throwable + { + // simple column identifiers with aliases are represented in ResultSet.Metadata + // by a ColumnSpecification based on the underlying ColumnDefinition + ColumnSpecification kSpec = columnSpecification("k_alias", Int32Type.instance); + ColumnSpecification v1Spec = columnSpecification("v1_alias", Int32Type.instance); + ColumnSpecification v2Spec = columnSpecification("v2_alias", AsciiType.instance); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(kSpec, columnDefinition("k")) - .addMapping(v1Spec, columnDefinition("v1")) - .addMapping(v2Spec, columnDefinition("v2")); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(kSpec, columnDefinition("k")) ++ .addMapping(v1Spec, columnDefinition("v1")) ++ .addMapping(v2Spec, columnDefinition("v2")); ++ ++ assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_alias FROM %s")); ++ } ++ ++ private void testUserTypes() throws Throwable ++ { ++ // User type fields are represented in ResultSet.Metadata by a ++ // ColumnSpecification denoting the name and type of the particular field ++ ColumnSpecification f1Spec = columnSpecification("v3.f1", Int32Type.instance); ++ ColumnSpecification f2Spec = columnSpecification("v3.f2", UTF8Type.instance); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(f1Spec, columnDefinition("v3")) ++ .addMapping(f2Spec, columnDefinition("v3")); + - assertEquals(expected, extractColumnMappingFromSelect( - "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_alias FROM %s")); ++ assertEquals(expected, extractColumnMappingFromSelect("SELECT v3.f1, v3.f2 FROM %s")); ++ } ++ ++ private void testUserTypesWithAliases() throws Throwable ++ { ++ // User type fields with aliases are represented in ResultSet.Metadata ++ // by a ColumnSpecification with the alias name and the type of the actual field ++ ColumnSpecification f1Spec = columnSpecification("f1_alias", Int32Type.instance); ++ ColumnSpecification f2Spec = columnSpecification("f2_alias", UTF8Type.instance); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(f1Spec, columnDefinition("v3")) ++ .addMapping(f2Spec, columnDefinition("v3")); ++ ++ assertEquals(expected, extractColumnMappingFromSelect("SELECT v3.f1 AS f1_alias, v3.f2 AS f2_alias FROM %s")); + } + + private void testWritetimeAndTTL() throws Throwable + { + // writetime and ttl are represented in ResultSet.Metadata by a ColumnSpecification + // with the function name plus argument and a long or int type respectively + ColumnSpecification wtSpec = columnSpecification("writetime(v1)", LongType.instance); + ColumnSpecification ttlSpec = columnSpecification("ttl(v2)", Int32Type.instance); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(wtSpec, columnDefinition("v1")) - .addMapping(ttlSpec, columnDefinition("v2")); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(wtSpec, columnDefinition("v1")) ++ .addMapping(ttlSpec, columnDefinition("v2")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT writetime(v1), ttl(v2) FROM %s")); + } + + private void testWritetimeAndTTLWithAliases() throws Throwable + { + // writetime and ttl with aliases are represented in ResultSet.Metadata + // by a ColumnSpecification with the alias name and the appropriate numeric type + ColumnSpecification wtSpec = columnSpecification("wt_alias", LongType.instance); + ColumnSpecification ttlSpec = columnSpecification("ttl_alias", Int32Type.instance); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(wtSpec, columnDefinition("v1")) - .addMapping(ttlSpec, columnDefinition("v2")); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(wtSpec, columnDefinition("v1")) ++ .addMapping(ttlSpec, columnDefinition("v2")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT writetime(v1) AS wt_alias, ttl(v2) AS ttl_alias FROM %s")); + } + + private void testFunction() throws Throwable + { + // a function such as intasblob(<col>) is represented in ResultSet.Metadata + // by a ColumnSpecification with the function name plus args and the type set + // to the function's return type + ColumnSpecification fnSpec = columnSpecification("intasblob(v1)", BytesType.instance); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(fnSpec, columnDefinition("v1")); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(fnSpec, columnDefinition("v1")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT intasblob(v1) FROM %s")); + } + + private void testFunctionWithAlias() throws Throwable + { + // a function with an alias is represented in ResultSet.Metadata by a + // ColumnSpecification with the alias and the type set to the function's + // return type + ColumnSpecification fnSpec = columnSpecification("fn_alias", BytesType.instance); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(fnSpec, columnDefinition("v1")); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(fnSpec, columnDefinition("v1")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT intasblob(v1) AS fn_alias FROM %s")); + } + + private void testMultipleAliasesOnSameColumn() throws Throwable + { + // Multiple result columns derived from the same underlying column are + // represented by ColumnSpecifications + ColumnSpecification alias1 = columnSpecification("alias_1", Int32Type.instance); + ColumnSpecification alias2 = columnSpecification("alias_2", Int32Type.instance); - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(alias1, columnDefinition("v1")) - .addMapping(alias2, columnDefinition("v1")); ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(alias1, columnDefinition("v1")) ++ .addMapping(alias2, columnDefinition("v1")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT v1 AS alias_1, v1 AS alias_2 FROM %s")); + } + + private void testMixedColumnTypes() throws Throwable + { + ColumnSpecification kSpec = columnSpecification("k_alias", Int32Type.instance); + ColumnSpecification v1Spec = columnSpecification("writetime(v1)", LongType.instance); + ColumnSpecification v2Spec = columnSpecification("ttl_alias", Int32Type.instance); - - SelectionColumns expected = SelectionColumnMapping.newMapping() - .addMapping(kSpec, columnDefinition("k")) - .addMapping(v1Spec, columnDefinition("v1")) - .addMapping(v2Spec, columnDefinition("v2")); ++ ColumnSpecification f1Spec = columnSpecification("v3.f1", Int32Type.instance); ++ ColumnSpecification f2Spec = columnSpecification("f2_alias", UTF8Type.instance); ++ ++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping() ++ .addMapping(kSpec, columnDefinition("k")) ++ .addMapping(v1Spec, columnDefinition("v1")) ++ .addMapping(v2Spec, columnDefinition("v2")) ++ .addMapping(f1Spec, columnDefinition("v3")) ++ .addMapping(f2Spec, columnDefinition("v3")) ++ .addMapping(columnDefinition("v3"), columnDefinition( ++ "v3")); + + assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS k_alias," + + " writetime(v1)," + - " ttl(v2) as ttl_alias" + ++ " ttl(v2) as ttl_alias," + ++ " v3.f1," + ++ " v3.f2 AS f2_alias," + ++ " v3" + + " FROM %s")); + } + + private SelectionColumns extractColumnMappingFromSelect(String query) throws RequestValidationException + { + CQLStatement statement = QueryProcessor.getStatement(String.format(query, KEYSPACE + "." + tableName), + ClientState.forInternalCalls()).statement; + assertTrue(statement instanceof SelectStatement); + return ((SelectStatement)statement).getSelection().getColumnMapping(); + } + - private CFDefinition.Name columnDefinition(String name) ++ private ColumnDefinition columnDefinition(String name) + { + return Schema.instance.getCFMetaData(KEYSPACE, tableName) - .getCfDef() - .get(new ColumnIdentifier(name, true)); ++ .getColumnDefinition(new ColumnIdentifier(name, true)); + + } + - private Iterable<CFDefinition.Name> columnDefinitions(String...name) ++ private Iterable<ColumnDefinition> columnDefinitions(String...name) + { - List<CFDefinition.Name> list = new ArrayList<>(); ++ List<ColumnDefinition> list = new ArrayList<>(); + for (String n : name) + list.add(columnDefinition(n)); + return list; + } + + private ColumnSpecification columnSpecification(String name, AbstractType<?> type) + { + return new ColumnSpecification(KEYSPACE, + tableName, + new ColumnIdentifier(name, true), + type); + } - - private void createTable(String query) throws Throwable - { - executeSchemaChange(String.format(query, KEYSPACE + "." + tableName)); - } - - private static void executeSchemaChange(String query) throws Throwable - { - try - { - process(query, ConsistencyLevel.ONE); - } - catch (RuntimeException exc) - { - throw exc.getCause(); - } - } + }