You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sa...@apache.org on 2015/06/18 18:51:17 UTC
[01/12] cassandra git commit: Expose some internals of
SelectStatement for inspection by QueryHandlers
Repository: cassandra
Updated Branches:
refs/heads/cassandra-2.0 ad8047abd -> f32cff8e1
refs/heads/cassandra-2.1 9966419db -> 0452e74f5
refs/heads/cassandra-2.2 a13399279 -> 1f8516da8
refs/heads/trunk 67baaef0f -> 40424ee59
Expose some internals of SelectStatement for inspection by QueryHandlers
patch by Sam Tunnicliffe; reviewed by Benjamin Lerer and Mick Semb Wever
for CASSANDRA-9532
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f32cff8e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f32cff8e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f32cff8e
Branch: refs/heads/cassandra-2.0
Commit: f32cff8e1fb69317219ffaee81b5861a54b83a1b
Parents: ad8047a
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 4 18:12:35 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jun 18 17:11:00 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../cassandra/cql3/ColumnSpecification.java | 22 ++
.../cql3/statements/SelectStatement.java | 37 ++-
.../cassandra/cql3/statements/Selection.java | 80 +++---
.../cql3/statements/SelectionColumnMapping.java | 106 ++++++++
.../cql3/statements/SelectionColumns.java | 19 ++
.../statements/SelectionColumnMappingTest.java | 244 +++++++++++++++++++
7 files changed, 476 insertions(+), 33 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 753fb1c..a235528 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.0.16:
+ * 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/f32cff8e/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
index 4dae701..089a1c5 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
@@ -17,6 +17,8 @@
*/
package org.apache.cassandra.cql3;
+import com.google.common.base.Objects;
+
import org.apache.cassandra.db.marshal.AbstractType;
public class ColumnSpecification
@@ -40,4 +42,24 @@ public class ColumnSpecification
// 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/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 95e0441..1c19760 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -60,7 +60,10 @@ 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
{
@@ -184,6 +187,14 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return boundTerms;
}
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public Selection getSelection()
+ {
+ return selection;
+ }
+
public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
{
state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT);
@@ -580,7 +591,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return new SliceQueryFilter(slices, isReversed, limit, toGroup);
}
- private int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
{
int l = Integer.MAX_VALUE;
if (limit != null)
@@ -1067,6 +1081,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
variables);
}
+ /**
+ * May be used by custom QueryHandler implementations
+ */
public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
{
if (!usesSecondaryIndexing || restrictedNames.isEmpty())
@@ -1446,7 +1463,21 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return true;
}
- private boolean hasClusteringColumnsRestriction()
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public boolean hasPartitionKeyRestriction()
+ {
+ for (int i = 0; i < keyRestrictions.length; i++)
+ if (keyRestrictions[i] != null)
+ return true;
+ return false;
+ }
+
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public boolean hasClusteringColumnsRestriction()
{
for (int i = 0; i < columnRestrictions.length; i++)
if (columnRestrictions[i] != null)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index 223f698..50a34bf 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -18,8 +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 org.apache.cassandra.cql3.*;
import org.apache.cassandra.cql3.functions.Function;
@@ -37,14 +38,14 @@ import org.apache.cassandra.utils.ByteBufferUtil;
public abstract class Selection
{
private final List<CFDefinition.Name> columns;
- private final List<ColumnSpecification> metadata;
+ private final SelectionColumns columnMapping;
private final boolean collectTimestamps;
private final boolean collectTTLs;
- protected Selection(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
+ protected Selection(List<CFDefinition.Name> columns, SelectionColumns columnMapping, boolean collectTimestamps, boolean collectTTLs)
{
this.columns = columns;
- this.metadata = metadata;
+ this.columnMapping = columnMapping;
this.collectTimestamps = collectTimestamps;
this.collectTTLs = collectTTLs;
}
@@ -57,7 +58,7 @@ public abstract class Selection
public ResultSet.Metadata getResultMetadata()
{
- return new ResultSet.Metadata(metadata);
+ return new ResultSet.Metadata(columnMapping.getColumnSpecifications());
}
public static Selection wildcard(CFDefinition cfDef)
@@ -94,21 +95,28 @@ public abstract class Selection
return idx;
}
- private static Selector makeSelector(CFDefinition cfDef, RawSelector raw, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+ private static Selector makeSelector(CFDefinition cfDef,
+ RawSelector raw,
+ List<CFDefinition.Name> names,
+ SelectionColumnMapping columnMapping) throws InvalidRequestException
{
Selectable selectable = raw.selectable.prepare(cfDef.cfm);
- return makeSelector(cfDef, selectable, raw.alias, names, metadata);
+ return makeSelector(cfDef, selectable, raw.alias, names, columnMapping);
}
- private static Selector makeSelector(CFDefinition cfDef, Selectable selectable, ColumnIdentifier alias, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+ private static Selector makeSelector(CFDefinition cfDef,
+ Selectable selectable,
+ ColumnIdentifier alias,
+ List<CFDefinition.Name> names,
+ SelectionColumnMapping columnMapping) throws InvalidRequestException
{
if (selectable instanceof ColumnIdentifier)
{
- CFDefinition.Name name = cfDef.get((ColumnIdentifier)selectable);
+ CFDefinition.Name name = cfDef.get((ColumnIdentifier) selectable);
if (name == null)
throw new InvalidRequestException(String.format("Undefined name %s in selection clause", selectable));
- if (metadata != null)
- metadata.add(alias == null ? name : makeAliasSpec(cfDef, name.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);
}
else if (selectable instanceof Selectable.WritetimeOrTTL)
@@ -121,25 +129,26 @@ public abstract class Selection
throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", name));
if (name.type.isCollection())
throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl"));
-
- if (metadata != null)
- metadata.add(makeWritetimeOrTTLSpec(cfDef, tot, alias));
+ if (columnMapping != null)
+ columnMapping.addMapping(makeWritetimeOrTTLSpec(cfDef, tot, alias), name);
return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime);
}
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
+ SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
for (Selectable rawArg : withFun.args)
- args.add(makeSelector(cfDef, rawArg, null, names, null));
+ args.add(makeSelector(cfDef, rawArg, null, names, tmpMapping));
AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfDef.cfm.ksName, cfDef.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);
- if (metadata != null)
- metadata.add(spec);
+ if (columnMapping != null)
+ columnMapping.addMapping(spec, tmpMapping.getMappings().values());
return new FunctionSelector(fun, args);
}
}
@@ -178,23 +187,23 @@ public abstract class Selection
if (needsProcessing)
{
List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>();
- 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(cfDef, rawSelector, names, metadata);
+ Selector selector = makeSelector(cfDef, rawSelector, names, columnMapping);
selectors.add(selector);
collectTimestamps |= selector.usesTimestamps();
collectTTLs |= selector.usesTTLs();
}
- return new SelectionWithProcessing(names, metadata, selectors, collectTimestamps, collectTTLs);
+ return new SelectionWithProcessing(names, columnMapping, selectors, collectTimestamps, collectTTLs);
}
else
{
List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(rawSelectors.size());
- List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
+ SelectionColumnMapping columnMapping = SelectionColumnMapping.newMapping();
for (RawSelector rawSelector : rawSelectors)
{
assert rawSelector.selectable instanceof ColumnIdentifier.Raw;
@@ -203,9 +212,12 @@ public abstract class Selection
if (name == null)
throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
names.add(name);
- metadata.add(rawSelector.alias == null ? name : makeAliasSpec(cfDef, name.type, rawSelector.alias));
+ columnMapping.addMapping(rawSelector.alias == null ? name : makeAliasSpec(cfDef,
+ name.type,
+ rawSelector.alias),
+ name);
}
- return new SimpleSelection(names, metadata, false);
+ return new SimpleSelection(names, columnMapping, false);
}
}
@@ -233,6 +245,14 @@ public abstract class Selection
return columns;
}
+ /**
+ * @return the mappings between resultset columns and the underlying columns
+ */
+ public SelectionColumns getColumnMapping()
+ {
+ return columnMapping;
+ }
+
public ResultSetBuilder resultSetBuilder(long now)
{
return new ResultSetBuilder(now);
@@ -264,7 +284,7 @@ public abstract class Selection
private ResultSetBuilder(long now)
{
- this.resultSet = new ResultSet(metadata);
+ this.resultSet = new ResultSet(columnMapping.getColumnSpecifications());
this.timestamps = collectTimestamps ? new long[columns.size()] : null;
this.ttls = collectTTLs ? new int[columns.size()] : null;
this.now = now;
@@ -321,17 +341,17 @@ public abstract class Selection
public SimpleSelection(List<CFDefinition.Name> columns, boolean isWildcard)
{
- this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
+ this(columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
}
- public SimpleSelection(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean isWildcard)
+ public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping, boolean isWildcard)
{
/*
* In theory, even a simple selection could have multiple time the same column, so we
* could filter those duplicate out of columns. But since we're very unlikely to
* get much duplicate in practice, it's more efficient not to bother.
*/
- super(columns, metadata, false, false);
+ super(columns, columnMapping, false, false);
this.isWildcard = isWildcard;
}
@@ -351,9 +371,9 @@ public abstract class Selection
{
private final List<Selector> selectors;
- public SelectionWithProcessing(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
+ public SelectionWithProcessing(List<CFDefinition.Name> columns, SelectionColumns columnMapping, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
{
- super(columns, metadata, collectTimestamps, collectTTLs);
+ super(columns, columnMapping, collectTimestamps, collectTTLs);
this.selectors = selectors;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
new file mode 100644
index 0000000..d09612f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
@@ -0,0 +1,106 @@
+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.collect.*;
+
+import org.apache.cassandra.cql3.CFDefinition;
+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 SelectionColumnMapping()
+ {
+ this.columnMappings = LinkedHashMultimap.create();
+ }
+
+ protected static SelectionColumnMapping newMapping()
+ {
+ return new SelectionColumnMapping();
+ }
+
+ protected static SelectionColumnMapping simpleMapping(List<CFDefinition.Name> columnDefinitions)
+ {
+ SelectionColumnMapping mapping = new SelectionColumnMapping();
+ for (CFDefinition.Name def: columnDefinitions)
+ mapping.addMapping(def, def);
+ return mapping;
+ }
+
+ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, CFDefinition.Name column)
+ {
+ columnMappings.put(colSpec, column);
+ return this;
+ }
+
+ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<CFDefinition.Name> 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());
+ }
+
+ public Multimap<ColumnSpecification, CFDefinition.Name> getMappings()
+ {
+ return Multimaps.unmodifiableMultimap(columnMappings);
+ }
+
+ public boolean equals(Object obj)
+ {
+ if (obj == null)
+ return false;
+
+ if (!(obj instanceof SelectionColumns))
+ return false;
+
+ return Objects.equals(columnMappings, ((SelectionColumns) obj).getMappings());
+ }
+
+ public int hashCode()
+ {
+ return Objects.hashCode(columnMappings);
+ }
+
+ public String toString()
+ {
+ final Function<CFDefinition.Name, String> getDefName = new Function<CFDefinition.Name, String>()
+ {
+ public String apply(CFDefinition.Name name)
+ {
+ return 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)
+ {
+ 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/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
new file mode 100644
index 0000000..3053f99
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
@@ -0,0 +1,19 @@
+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.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();
+}
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
new file mode 100644
index 0000000..9c31653
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
@@ -0,0 +1,244 @@
+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.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
+{
+ 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));
+ }
+
+ @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)");
+ testSimpleTypes();
+ testWildcard();
+ testSimpleTypesWithAliases();
+ 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)))");
+ ColumnSpecification tokenSpec = columnSpecification("token(a, b)", BytesType.instance);
+ SelectionColumns 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"));
+
+ 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"));
+
+ 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"));
+
+ assertEquals(expected, extractColumnMappingFromSelect(
+ "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS k_alias," +
+ " writetime(v1)," +
+ " ttl(v2) as ttl_alias" +
+ " 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)
+ {
+ return Schema.instance.getCFMetaData(KEYSPACE, tableName)
+ .getCfDef()
+ .get(new ColumnIdentifier(name, true));
+
+ }
+
+ private Iterable<CFDefinition.Name> columnDefinitions(String...name)
+ {
+ List<CFDefinition.Name> 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();
+ }
+ }
+}
[10/12] cassandra git commit: Merge branch 'cassandra-2.1' into
cassandra-2.2
Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/Selector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selector.java
index 747dc60,0000000..9b7f0ba
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selector.java
@@@ -1,179 -1,0 +1,192 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+
+import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A <code>Selector</code> is used to convert the data returned by the storage engine into the data requested by the
+ * user. They correspond to the <selector> elements from the select clause.
+ * <p>Since the introduction of aggregation, <code>Selector</code>s cannot be called anymore by multiple threads
+ * as they have an internal state.</p>
+ */
+public abstract class Selector implements AssignmentTestable
+{
+ /**
+ * A factory for <code>Selector</code> instances.
+ */
+ public static abstract class Factory
+ {
+ public Iterable<Function> getFunctions()
+ {
+ return Collections.emptySet();
+ }
+
+ /**
+ * Returns the column specification corresponding to the output value of the selector instances created by
+ * this factory.
+ *
+ * @param cfm the column family meta data
+ * @return a column specification
+ */
+ public final ColumnSpecification getColumnSpecification(CFMetaData cfm)
+ {
+ return new ColumnSpecification(cfm.ksName,
+ cfm.cfName,
+ new ColumnIdentifier(getColumnName(), true),
+ getReturnType());
+ }
+
+ /**
+ * Creates a new <code>Selector</code> instance.
+ *
+ * @return a new <code>Selector</code> instance
+ */
+ public abstract Selector newInstance() throws InvalidRequestException;
+
+ /**
+ * Checks if this factory creates selectors instances that creates aggregates.
+ *
+ * @return <code>true</code> if this factory creates selectors instances that creates aggregates,
+ * <code>false</code> otherwise
+ */
+ public boolean isAggregateSelectorFactory()
+ {
+ return false;
+ }
+
+ /**
+ * Checks if this factory creates <code>writetime</code> selectors instances.
+ *
+ * @return <code>true</code> if this factory creates <code>writetime</code> selectors instances,
+ * <code>false</code> otherwise
+ */
+ public boolean isWritetimeSelectorFactory()
+ {
+ return false;
+ }
+
+ /**
+ * Checks if this factory creates <code>TTL</code> selectors instances.
+ *
+ * @return <code>true</code> if this factory creates <code>TTL</code> selectors instances,
+ * <code>false</code> otherwise
+ */
+ public boolean isTTLSelectorFactory()
+ {
+ return false;
+ }
+
+ /**
+ * Returns the name of the column corresponding to the output value of the selector instances created by
+ * this factory.
+ *
+ * @return a column name
+ */
+ protected abstract String getColumnName();
+
+ /**
+ * Returns the type of the values returned by the selector instances created by this factory.
+ *
+ * @return the selector output type
+ */
+ protected abstract AbstractType<?> getReturnType();
+
++ /**
++ * Record a mapping between the ColumnDefinitions that are used by the selector
++ * instances created by this factory and a column in the ResultSet.Metadata
++ * returned with a query. In most cases, this is likely to be a 1:1 mapping,
++ * but some selector instances may utilise multiple columns (or none at all)
++ * to produce a value (i.e. functions).
++ *
++ * @param mapping the instance of the column mapping belonging to the current query's Selection
++ * @param resultsColumn the column in the ResultSet.Metadata to which the ColumnDefinitions used
++ * by the Selector are to be mapped
++ */
++ protected abstract void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn);
+ }
+
+ /**
+ * Add the current value from the specified <code>ResultSetBuilder</code>.
+ *
+ * @param protocolVersion protocol version used for serialization
+ * @param rs the <code>ResultSetBuilder</code>
+ * @throws InvalidRequestException if a problem occurs while add the input value
+ */
+ public abstract void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException;
+
+ /**
+ * Returns the selector output.
+ *
+ * @param protocolVersion protocol version used for serialization
+ * @return the selector output
+ * @throws InvalidRequestException if a problem occurs while computing the output value
+ */
+ public abstract ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException;
+
+ /**
+ * Returns the <code>Selector</code> output type.
+ *
+ * @return the <code>Selector</code> output type.
+ */
+ public abstract AbstractType<?> getType();
+
+ /**
+ * Checks if this <code>Selector</code> is creating aggregates.
+ *
+ * @return <code>true</code> if this <code>Selector</code> is creating aggregates <code>false</code>
+ * otherwise.
+ */
+ public boolean isAggregate()
+ {
+ return false;
+ }
+
+ /**
+ * Reset the internal state of this <code>Selector</code>.
+ */
+ public abstract void reset();
+
+ public final AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+ {
+ // We should ignore the fact that the output type is frozen in our comparison as functions do not support
+ // frozen types for arguments
+ AbstractType<?> receiverType = receiver.type;
+ if (getType().isFrozenCollection())
+ receiverType = receiverType.freeze();
+
+ if (receiverType.equals(getType()))
+ return AssignmentTestable.TestResult.EXACT_MATCH;
+
+ if (receiverType.isValueCompatibleWith(getType()))
+ return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+ return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
index beb7399,0000000..81905e6
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
@@@ -1,206 -1,0 +1,206 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.util.*;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A set of <code>Selector</code> factories.
+ */
+final class SelectorFactories implements Iterable<Selector.Factory>
+{
+ /**
+ * The <code>Selector</code> factories.
+ */
+ private final List<Selector.Factory> factories;
+
+ /**
+ * <code>true</code> if one of the factory creates writetime selectors.
+ */
+ private boolean containsWritetimeFactory;
+
+ /**
+ * <code>true</code> if one of the factory creates TTL selectors.
+ */
+ private boolean containsTTLFactory;
+
+ /**
+ * The number of factories creating aggregates.
+ */
+ private int numberOfAggregateFactories;
+
+ /**
+ * Creates a new <code>SelectorFactories</code> instance and collect the column definitions.
+ *
+ * @param selectables the <code>Selectable</code>s for which the factories must be created
+ * @param cfm the Column Family Definition
+ * @param defs the collector parameter for the column definitions
+ * @return a new <code>SelectorFactories</code> instance
+ * @throws InvalidRequestException if a problem occurs while creating the factories
+ */
+ public static SelectorFactories createFactoriesAndCollectColumnDefinitions(List<Selectable> selectables,
+ CFMetaData cfm,
+ List<ColumnDefinition> defs)
+ throws InvalidRequestException
+ {
+ return new SelectorFactories(selectables, cfm, defs);
+ }
+
+ private SelectorFactories(List<Selectable> selectables,
+ CFMetaData cfm,
+ List<ColumnDefinition> defs)
+ throws InvalidRequestException
+ {
+ factories = new ArrayList<>(selectables.size());
+
+ for (Selectable selectable : selectables)
+ {
+ Factory factory = selectable.newSelectorFactory(cfm, defs);
+ containsWritetimeFactory |= factory.isWritetimeSelectorFactory();
+ containsTTLFactory |= factory.isTTLSelectorFactory();
+ if (factory.isAggregateSelectorFactory())
+ ++numberOfAggregateFactories;
+ factories.add(factory);
+ }
+ }
+
+ public Iterable<Function> getFunctions()
+ {
+ Iterable<Function> functions = Collections.emptySet();
+ for (Factory factory : factories)
+ if (factory != null)
+ functions = Iterables.concat(functions, factory.getFunctions());
+ return functions;
+ }
+
+ /**
+ * Adds a new <code>Selector.Factory</code> for a column that is needed only for ORDER BY purposes.
+ * @param def the column that is needed for ordering
+ * @param index the index of the column definition in the Selection's list of columns
+ */
+ public void addSelectorForOrdering(ColumnDefinition def, int index)
+ {
- factories.add(SimpleSelector.newFactory(def.name.toString(), index, def.type));
++ factories.add(SimpleSelector.newFactory(def, index));
+ }
+
+ /**
+ * Checks if this <code>SelectorFactories</code> contains only factories for aggregates.
+ *
+ * @return <code>true</code> if this <code>SelectorFactories</code> contains only factories for aggregates,
+ * <code>false</code> otherwise.
+ */
+ public boolean containsOnlyAggregateFunctions()
+ {
+ int size = factories.size();
+ return size != 0 && numberOfAggregateFactories == size;
+ }
+
+ /**
+ * Whether the selector built by this factory does aggregation or not (either directly or in a sub-selector).
+ *
+ * @return <code>true</code> if the selector built by this factor does aggregation, <code>false</code> otherwise.
+ */
+ public boolean doesAggregation()
+ {
+ return numberOfAggregateFactories > 0;
+ }
+
+ /**
+ * Checks if this <code>SelectorFactories</code> contains at least one factory for writetime selectors.
+ *
+ * @return <code>true</code> if this <code>SelectorFactories</code> contains at least one factory for writetime
+ * selectors, <code>false</code> otherwise.
+ */
+ public boolean containsWritetimeSelectorFactory()
+ {
+ return containsWritetimeFactory;
+ }
+
+ /**
+ * Checks if this <code>SelectorFactories</code> contains at least one factory for TTL selectors.
+ *
+ * @return <code>true</code> if this <code>SelectorFactories</code> contains at least one factory for TTL
+ * selectors, <code>false</code> otherwise.
+ */
+ public boolean containsTTLSelectorFactory()
+ {
+ return containsTTLFactory;
+ }
+
+ /**
+ * Creates a list of new <code>Selector</code> instances.
+ * @return a list of new <code>Selector</code> instances.
+ */
+ public List<Selector> newInstances() throws InvalidRequestException
+ {
+ List<Selector> selectors = new ArrayList<>(factories.size());
+ for (Selector.Factory factory : factories)
+ {
+ selectors.add(factory.newInstance());
+ }
+ return selectors;
+ }
+
+ public Iterator<Factory> iterator()
+ {
+ return factories.iterator();
+ }
+
+ /**
+ * Returns the names of the columns corresponding to the output values of the selector instances created by
+ * these factories.
+ *
+ * @return a list of column names
+ */
+ public List<String> getColumnNames()
+ {
+ return Lists.transform(factories, new com.google.common.base.Function<Selector.Factory, String>()
+ {
+ public String apply(Selector.Factory factory)
+ {
+ return factory.getColumnName();
+ }
+ });
+ }
+
+ /**
+ * Returns a list of the return types of the selector instances created by these factories.
+ *
+ * @return a list of types
+ */
+ public List<AbstractType<?>> getReturnTypes()
+ {
+ return Lists.transform(factories, new com.google.common.base.Function<Selector.Factory, AbstractType<?>>()
+ {
+ public AbstractType<?> apply(Selector.Factory factory)
+ {
+ return factory.getReturnType();
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
index c2edaed,0000000..6c4dc04
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
@@@ -1,93 -1,0 +1,100 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+
++import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public final class SimpleSelector extends Selector
+{
+ private final String columnName;
+ private final int idx;
+ private final AbstractType<?> type;
+ private ByteBuffer current;
+
- public static Factory newFactory(final String columnName, final int idx, final AbstractType<?> type)
++ public static Factory newFactory(final ColumnDefinition def, final int idx)
+ {
+ return new Factory()
+ {
+ @Override
+ protected String getColumnName()
+ {
- return columnName;
++ return def.name.toString();
+ }
+
+ @Override
+ protected AbstractType<?> getReturnType()
+ {
- return type;
++ return def.type;
++ }
++
++ protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultColumn)
++ {
++ mapping.addMapping(resultColumn, def);
+ }
+
+ @Override
+ public Selector newInstance()
+ {
- return new SimpleSelector(columnName, idx, type);
++ return new SimpleSelector(def.name.toString(), idx, def.type);
+ }
+ };
+ }
+
+ @Override
+ public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+ {
+ current = rs.current.get(idx);
+ }
+
+ @Override
+ public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+ {
+ return current;
+ }
+
+ @Override
+ public void reset()
+ {
+ current = null;
+ }
+
+ @Override
+ public AbstractType<?> getType()
+ {
+ return type;
+ }
+
+ @Override
+ public String toString()
+ {
+ return columnName;
+ }
+
+ private SimpleSelector(String columnName, int idx, AbstractType<?> type)
+ {
+ this.columnName = columnName;
+ this.idx = idx;
+ this.type = type;
+ }
- }
++}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
index a1ecd3d,0000000..b3607f3
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
@@@ -1,108 -1,0 +1,116 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
++import java.util.Collections;
+
++import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+final class WritetimeOrTTLSelector extends Selector
+{
+ private final String columnName;
+ private final int idx;
+ private final boolean isWritetime;
+ private ByteBuffer current;
+
- public static Factory newFactory(final String columnName, final int idx, final boolean isWritetime)
++ public static Factory newFactory(final ColumnDefinition def, final int idx, final boolean isWritetime)
+ {
+ return new Factory()
+ {
+ protected String getColumnName()
+ {
- return String.format("%s(%s)", isWritetime ? "writetime" : "ttl", columnName);
++ return String.format("%s(%s)", isWritetime ? "writetime" : "ttl", def.name.toString());
+ }
+
+ protected AbstractType<?> getReturnType()
+ {
+ return isWritetime ? LongType.instance : Int32Type.instance;
+ }
+
++ protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
++ {
++ mapping.addMapping(resultsColumn, def);
++ }
++
+ public Selector newInstance()
+ {
- return new WritetimeOrTTLSelector(columnName, idx, isWritetime);
++ return new WritetimeOrTTLSelector(def.name.toString(), idx, isWritetime);
+ }
+
+ public boolean isWritetimeSelectorFactory()
+ {
+ return isWritetime;
+ }
+
+ public boolean isTTLSelectorFactory()
+ {
+ return !isWritetime;
+ }
+ };
+ }
+
+ public void addInput(int protocolVersion, ResultSetBuilder rs)
+ {
+ if (isWritetime)
+ {
+ long ts = rs.timestamps[idx];
+ current = ts != Long.MIN_VALUE ? ByteBufferUtil.bytes(ts) : null;
+ }
+ else
+ {
+ int ttl = rs.ttls[idx];
+ current = ttl > 0 ? ByteBufferUtil.bytes(ttl) : null;
+ }
+ }
+
+ public ByteBuffer getOutput(int protocolVersion)
+ {
+ return current;
+ }
+
+ public void reset()
+ {
+ current = null;
+ }
+
+ public AbstractType<?> getType()
+ {
+ return isWritetime ? LongType.instance : Int32Type.instance;
+ }
+
+ @Override
+ public String toString()
+ {
+ return columnName;
+ }
+
+ private WritetimeOrTTLSelector(String columnName, int idx, boolean isWritetime)
+ {
+ this.columnName = columnName;
+ this.idx = idx;
+ this.isWritetime = isWritetime;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index dfb0d07,d0566eb..8ce555f
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -288,9 -341,11 +292,25 @@@ public class SelectStatement implement
return cfm.cfName;
}
++ /**
++ * May be used by custom QueryHandler implementations
++ */
++ public Selection getSelection()
++ {
++ return selection;
++ }
++
++ /**
++ * May be used by custom QueryHandler implementations
++ */
++ public StatementRestrictions getRestrictions()
++ {
++ return restrictions;
++ }
++
private List<ReadCommand> getSliceCommands(QueryOptions options, int limit, long now) throws RequestValidationException
{
- Collection<ByteBuffer> keys = getKeys(options);
- if (keys.isEmpty()) // in case of IN () for (the last column of) the partition key.
- return null;
+ Collection<ByteBuffer> keys = restrictions.getPartitionKeys(options);
List<ReadCommand> commands = new ArrayList<>(keys.size());
@@@ -453,14 -568,18 +473,17 @@@
return new SliceQueryFilter(slices, isReversed, limit, toGroup);
}
- private int getLimit(QueryOptions options) throws InvalidRequestException
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public int getLimit(QueryOptions options) throws InvalidRequestException
{
- int l = Integer.MAX_VALUE;
if (limit != null)
{
- ByteBuffer b = limit.bindAndGet(options);
- if (b == null)
- throw new InvalidRequestException("Invalid null value of limit");
-
+ ByteBuffer b = checkNotNull(limit.bindAndGet(options), "Invalid null value of limit");
+ // treat UNSET limit value as 'unlimited'
+ if (b == UNSET_BYTE_BUFFER)
+ return Integer.MAX_VALUE;
try
{
Int32Type.instance.validate(b);
@@@ -530,18 -853,300 +553,21 @@@
}
}
- /** Returns true if a non-frozen collection is selected, false otherwise. */
- private boolean selectACollection()
- {
- if (!cfm.comparator.hasCollections())
- return false;
-
- for (ColumnDefinition def : selection.getColumns())
- {
- if (def.type.isCollection() && def.type.isMultiCell())
- return true;
- }
-
- return false;
- }
-
- @VisibleForTesting
- static List<Composite> buildBound(Bound bound,
- List<ColumnDefinition> defs,
- Restriction[] restrictions,
- boolean isReversed,
- CType type,
- QueryOptions options) throws InvalidRequestException
- {
- CBuilder builder = type.builder();
-
- // 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),
- // it only depends on whether the slice is reversed
- Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
- for (int i = 0, m = defs.size(); i < m; i++)
- {
- ColumnDefinition def = defs.get(i);
-
- // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
- // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
- // But if the actual comparator itself is reversed, we must inversed the bounds too.
- Bound b = isReversed == isReversedType(def) ? bound : Bound.reverse(bound);
- Restriction r = restrictions[def.position()];
- if (isNullRestriction(r, b) || !r.canEvaluateWithSlices())
- {
- // 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.
- Composite prefix = builder.build();
- return Collections.singletonList(eocBound == Bound.END ? prefix.end() : prefix.start());
- }
- if (r.isSlice())
- {
- if (r.isMultiColumn())
- {
- MultiColumnRestriction.Slice slice = (MultiColumnRestriction.Slice) r;
-
- if (!slice.hasBound(b))
- {
- Composite prefix = builder.build();
- return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
- ? prefix.end()
- : prefix);
- }
-
- List<ByteBuffer> vals = slice.componentBounds(b, options);
-
- for (int j = 0, n = vals.size(); j < n; j++)
- addValue(builder, defs.get(i + j), vals.get(j)) ;
- }
- else
- {
- builder.add(getSliceValue(r, b, options));
- }
- Operator relType = ((Restriction.Slice)r).getRelation(eocBound, b);
- return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
- }
-
- if (r.isIN())
- {
- // 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> inValues = new TreeSet<>(isReversed ? type.reverseComparator() : type);
-
- if (r.isMultiColumn())
- {
- List<List<ByteBuffer>> splitInValues = ((MultiColumnRestriction.IN) r).splitValues(options);
-
- for (List<ByteBuffer> components : splitInValues)
- {
- for (int j = 0; j < components.size(); j++)
- if (components.get(j) == null)
- throw new InvalidRequestException("Invalid null value in condition for column " + defs.get(i + j).name);
-
- Composite prefix = builder.buildWith(components);
- inValues.add(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
- }
- return new ArrayList<>(inValues);
- }
-
- List<ByteBuffer> values = r.values(options);
- if (values.size() != 1)
- {
- // IN query, we only support it on the clustering columns
- assert def.position() == defs.size() - 1;
- for (ByteBuffer val : values)
- {
- if (val == null)
- throw new InvalidRequestException(String.format("Invalid null value in condition for column %s",
- def.name));
- Composite prefix = builder.buildWith(val);
- // See below for why this
- inValues.add(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
- }
- return new ArrayList<>(inValues);
- }
- }
-
- List<ByteBuffer> values = r.values(options);
-
- if (r.isMultiColumn())
- {
- for (int j = 0; j < values.size(); j++)
- addValue(builder, defs.get(i + j), values.get(j));
- i += values.size() - 1; // skips the processed columns
- }
- else
- {
- addValue(builder, def, values.get(0));
- }
- }
- // Means no relation at all or everything was an equal
- // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
- // 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
- // prefix.end() will sort after prefix (see #5240).
- Composite prefix = builder.build();
- return Collections.singletonList(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
- }
-
- /**
- * Adds an EOC to the specified Composite.
- *
- * @param composite the composite
- * @param eocBound the EOC bound
- * @return a new <code>Composite</code> with the EOC corresponding to the eocBound
- */
- private static Composite addEOC(Composite composite, Bound eocBound)
- {
- return eocBound == Bound.END ? composite.end() : composite.start();
- }
-
- /**
- * Adds the specified value to the specified builder
- *
- * @param builder the CBuilder to which the value must be added
- * @param def the column associated to the value
- * @param value the value to add
- * @throws InvalidRequestException if the value is null
- */
- private static void addValue(CBuilder builder, ColumnDefinition def, ByteBuffer value) throws InvalidRequestException
- {
- if (value == null)
- throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", def.name));
- builder.add(value);
- }
-
- private static Composite.EOC eocForRelation(Operator 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 static boolean isNullRestriction(Restriction r, Bound b)
- {
- return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
- }
-
- 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, options);
- if (val == null)
- throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
- return val;
- }
-
- private List<Composite> getRequestedBound(Bound b, QueryOptions options) throws InvalidRequestException
- {
- assert isColumnRange();
- return buildBound(b, cfm.clusteringColumns(), columnRestrictions, isReversed, cfm.comparator, options);
- }
-
+ /**
+ * May be used by custom QueryHandler implementations
+ */
public List<IndexExpression> getValidatedIndexExpressions(QueryOptions options) throws InvalidRequestException
{
- if (!usesSecondaryIndexing || restrictedColumns.isEmpty())
+ if (!restrictions.usesSecondaryIndexing())
return Collections.emptyList();
- List<IndexExpression> expressions = new ArrayList<IndexExpression>();
- for (ColumnDefinition def : restrictedColumns.keySet())
- {
- Restriction restriction;
- switch (def.kind)
- {
- case PARTITION_KEY:
- restriction = keyRestrictions[def.position()];
- break;
- case CLUSTERING_COLUMN:
- restriction = columnRestrictions[def.position()];
- break;
- case REGULAR:
- case STATIC:
- restriction = metadataRestrictions.get(def.name);
- break;
- default:
- // We don't allow restricting a COMPACT_VALUE for now in prepare.
- throw new AssertionError();
- }
-
- if (restriction.isSlice())
- {
- Restriction.Slice slice = (Restriction.Slice)restriction;
- for (Bound b : Bound.values())
- {
- if (slice.hasBound(b))
- {
- ByteBuffer value = validateIndexedValue(def, slice.bound(b, options));
- Operator op = slice.getIndexOperator(b);
- // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
- // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
- // use the underlying comparator as is.
- if (def.type instanceof ReversedType)
- op = reverse(op);
- expressions.add(new IndexExpression(def.name.bytes, op, value));
- }
- }
- }
- else if (restriction.isContains())
- {
- SingleColumnRestriction.Contains contains = (SingleColumnRestriction.Contains)restriction;
- for (ByteBuffer value : contains.values(options))
- {
- validateIndexedValue(def, value);
- expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS, value));
- }
- for (ByteBuffer key : contains.keys(options))
- {
- validateIndexedValue(def, key);
- expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS_KEY, key));
- }
- }
- else
- {
- ByteBuffer value;
- if (restriction.isMultiColumn())
- {
- List<ByteBuffer> values = restriction.values(options);
- value = values.get(def.position());
- }
- else
- {
- List<ByteBuffer> values = restriction.values(options);
- if (values.size() != 1)
- throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
+ SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
- value = values.get(0);
- }
+ List<IndexExpression> expressions = restrictions.getIndexExpressions(secondaryIndexManager, options);
- validateIndexedValue(def, value);
- expressions.add(new IndexExpression(def.name.bytes, Operator.EQ, value));
- }
- }
+ secondaryIndexManager.validateIndexSearchersForQuery(expressions);
- if (usesSecondaryIndexing)
- {
- ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
- SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
- secondaryIndexManager.validateIndexSearchersForQuery(expressions);
- }
-
return expressions;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
index 0000000,0000000..5bacf0d
new file mode 100644
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
@@@ -1,0 -1,0 +1,353 @@@
++package org.apache.cassandra.cql3.selection;
++
++import java.util.Collections;
++
++import com.google.common.collect.ImmutableList;
++import org.junit.Test;
++
++import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.Schema;
++import org.apache.cassandra.cql3.*;
++import org.apache.cassandra.cql3.statements.SelectStatement;
++import org.apache.cassandra.db.marshal.*;
++import org.apache.cassandra.exceptions.RequestValidationException;
++import org.apache.cassandra.service.ClientState;
++import org.apache.cassandra.utils.ByteBufferUtil;
++
++import static org.junit.Assert.assertEquals;
++import static org.junit.Assert.assertFalse;
++import static org.junit.Assert.assertTrue;
++
++public class SelectionColumnMappingTest extends CQLTester
++{
++ String tableName;
++ String typeName;
++ UserType userType;
++ String functionName;
++
++ @Test
++ public void testSelectionColumnMapping() throws Throwable
++ {
++ // Organised as a single test to avoid the overhead of
++ // table creation for each variant
++
++ 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 + ">)");
++ userType = Schema.instance.getKSMetaData(KEYSPACE).userTypes.getType(ByteBufferUtil.bytes(typeName));
++ functionName = createFunction(KEYSPACE, "int, ascii",
++ "CREATE FUNCTION %s (i int, a ascii) " +
++ "CALLED ON NULL INPUT " +
++ "RETURNS int " +
++ "LANGUAGE java " +
++ "AS 'return Integer.valueOf(i);'");
++ testSimpleTypes();
++ testWildcard();
++ testSimpleTypesWithAliases();
++ testUserTypes();
++ testUserTypesWithAliases();
++ testWritetimeAndTTL();
++ testWritetimeAndTTLWithAliases();
++ testFunction();
++ testNoArgFunction();
++ testUserDefinedFunction();
++ testOverloadedFunction();
++ testFunctionWithAlias();
++ testMultipleAliasesOnSameColumn();
++ testCount();
++ testMixedColumnTypes();
++ }
++
++ @Test
++ public void testMultipleArgumentFunction() throws Throwable
++ {
++ // demonstrate behaviour of token() with composite partition key
++ tableName = createTable("CREATE TABLE %s (a int, b text, PRIMARY KEY ((a, b)))");
++ ColumnSpecification tokenSpec = columnSpecification("system.token(a, b)", BytesType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(tokenSpec, columnDefinition("a"))
++ .addMapping(tokenSpec, columnDefinition("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
++ ColumnSpecification kSpec = columnSpecification("k", Int32Type.instance);
++ ColumnSpecification v1Spec = columnSpecification("v1", Int32Type.instance);
++ ColumnSpecification v2Spec = columnSpecification("v2", AsciiType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(kSpec, columnDefinition("k"))
++ .addMapping(v1Spec, columnDefinition("v1"))
++ .addMapping(v2Spec, columnDefinition("v2"));
++
++ assertEquals(expected, extractColumnMappingFromSelect("SELECT k, v1, v2 FROM %s"));
++ }
++
++ private void testWildcard() throws Throwable
++ {
++ // Wildcard select represents each column in the table with a ColumnDefinition
++ // in the ResultSet metadata
++ ColumnDefinition kSpec = columnDefinition("k");
++ ColumnDefinition v1Spec = columnDefinition("v1");
++ ColumnDefinition v2Spec = columnDefinition("v2");
++ ColumnDefinition v3Spec = columnDefinition("v3");
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(kSpec, columnDefinition("k"))
++ .addMapping(v1Spec, columnDefinition("v1"))
++ .addMapping(v2Spec, columnDefinition("v2"))
++ .addMapping(v3Spec, 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);
++ 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 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);
++ 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);
++ 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("system.intasblob(v1)", BytesType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(fnSpec, columnDefinition("v1"));
++
++ assertEquals(expected, extractColumnMappingFromSelect("SELECT intasblob(v1) FROM %s"));
++ }
++
++ private void testNoArgFunction() throws Throwable
++ {
++ // a no-arg function such as now() is represented in ResultSet.Metadata
++ // but has no mapping to any underlying column
++ ColumnSpecification fnSpec = columnSpecification("system.now()", TimeUUIDType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping().addMapping(fnSpec, null);
++
++ SelectionColumns actual = extractColumnMappingFromSelect("SELECT now() FROM %s");
++ assertEquals(expected, actual);
++ assertEquals(Collections.singletonList(fnSpec), actual.getColumnSpecifications());
++ assertTrue(actual.getMappings().isEmpty());
++ }
++
++ private void testOverloadedFunction() throws Throwable
++ {
++ String fnName = createFunction(KEYSPACE, "int",
++ "CREATE FUNCTION %s (input int) " +
++ "RETURNS NULL ON NULL INPUT " +
++ "RETURNS text " +
++ "LANGUAGE java " +
++ "AS 'return \"Hello World\";'");
++ createFunctionOverload(fnName, "text",
++ "CREATE FUNCTION %s (input text) " +
++ "RETURNS NULL ON NULL INPUT " +
++ "RETURNS text " +
++ "LANGUAGE java " +
++ "AS 'return \"Hello World\";'");
++
++ createFunctionOverload(fnName, "int, text",
++ "CREATE FUNCTION %s (input1 int, input2 text) " +
++ "RETURNS NULL ON NULL INPUT " +
++ "RETURNS text " +
++ "LANGUAGE java " +
++ "AS 'return \"Hello World\";'");
++ ColumnSpecification fnSpec1 = columnSpecification(fnName + "(v1)", UTF8Type.instance);
++ ColumnSpecification fnSpec2 = columnSpecification(fnName + "(v2)", UTF8Type.instance);
++ ColumnSpecification fnSpec3 = columnSpecification(fnName + "(v1, v2)", UTF8Type.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(fnSpec1, columnDefinition("v1"))
++ .addMapping(fnSpec2, columnDefinition("v2"))
++ .addMapping(fnSpec3, columnDefinition("v1"))
++ .addMapping(fnSpec3, columnDefinition("v2"));
++
++ String select = String.format("SELECT %1$s(v1), %1$s(v2), %1$s(v1, v2) FROM %%s", fnName);
++ SelectionColumns actual = extractColumnMappingFromSelect(select);
++
++ assertEquals(expected, actual);
++ assertEquals(ImmutableList.of(fnSpec1, fnSpec2, fnSpec3), actual.getColumnSpecifications());
++ }
++
++ private void testCount() throws Throwable
++ {
++ // SELECT COUNT does not necessarily include any mappings, but it must always return
++ // a singleton list from getColumnSpecifications() in order for the ResultSet.Metadata
++ // to be constructed correctly:
++ // * COUNT(*) / COUNT(1) do not generate any mappings, as no specific columns are referenced
++ // * COUNT(foo) does generate a mapping from the 'system.count' column spec to foo
++ ColumnSpecification count = columnSpecification("count", LongType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(count, null);
++
++ SelectionColumns actual = extractColumnMappingFromSelect("SELECT COUNT(*) FROM %s");
++ assertEquals(expected, actual);
++ assertEquals(Collections.singletonList(count), actual.getColumnSpecifications());
++ assertTrue(actual.getMappings().isEmpty());
++
++ actual = extractColumnMappingFromSelect("SELECT COUNT(1) FROM %s");
++ assertEquals(expected, actual);
++ assertEquals(Collections.singletonList(count), actual.getColumnSpecifications());
++ assertTrue(actual.getMappings().isEmpty());
++
++ ColumnSpecification countV1 = columnSpecification("system.count(v1)", LongType.instance);
++ expected = SelectionColumnMapping.newMapping()
++ .addMapping(countV1, columnDefinition("v1"));
++ actual = extractColumnMappingFromSelect("SELECT COUNT(v1) FROM %s");
++ assertEquals(expected, actual);
++ assertEquals(Collections.singletonList(countV1), actual.getColumnSpecifications());
++ assertFalse(actual.getMappings().isEmpty());
++ }
++
++ private void testUserDefinedFunction() throws Throwable
++ {
++ // UDFs are basically represented in the same way as system functions
++ String functionCall = String.format("%s(v1, v2)", functionName);
++ ColumnSpecification fnSpec = columnSpecification(functionCall, Int32Type.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(fnSpec, columnDefinition("v1"))
++ .addMapping(fnSpec, columnDefinition("v2"));
++
++ assertEquals(expected, extractColumnMappingFromSelect("SELECT " + functionCall + " 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);
++ 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);
++ 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);
++ ColumnSpecification f1Spec = columnSpecification("v3.f1", Int32Type.instance);
++ ColumnSpecification f2Spec = columnSpecification("f2_alias", UTF8Type.instance);
++ ColumnSpecification f3Spec = columnSpecification("v3", userType);
++
++ 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(f3Spec, columnDefinition("v3"));
++
++ assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS k_alias," +
++ " writetime(v1)," +
++ " 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 ColumnDefinition columnDefinition(String name)
++ {
++ return Schema.instance.getCFMetaData(KEYSPACE, tableName)
++ .getColumnDefinition(new ColumnIdentifier(name, true));
++
++ }
++
++ private ColumnSpecification columnSpecification(String name, AbstractType<?> type)
++ {
++ return new ColumnSpecification(KEYSPACE,
++ tableName,
++ new ColumnIdentifier(name, true),
++ type);
++ }
++}
[04/12] cassandra git commit: Expose some internals of
SelectStatement for inspection by QueryHandlers
Posted by sa...@apache.org.
Expose some internals of SelectStatement for inspection by QueryHandlers
patch by Sam Tunnicliffe; reviewed by Benjamin Lerer and Mick Semb Wever
for CASSANDRA-9532
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f32cff8e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f32cff8e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f32cff8e
Branch: refs/heads/trunk
Commit: f32cff8e1fb69317219ffaee81b5861a54b83a1b
Parents: ad8047a
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 4 18:12:35 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jun 18 17:11:00 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../cassandra/cql3/ColumnSpecification.java | 22 ++
.../cql3/statements/SelectStatement.java | 37 ++-
.../cassandra/cql3/statements/Selection.java | 80 +++---
.../cql3/statements/SelectionColumnMapping.java | 106 ++++++++
.../cql3/statements/SelectionColumns.java | 19 ++
.../statements/SelectionColumnMappingTest.java | 244 +++++++++++++++++++
7 files changed, 476 insertions(+), 33 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 753fb1c..a235528 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.0.16:
+ * 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/f32cff8e/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
index 4dae701..089a1c5 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
@@ -17,6 +17,8 @@
*/
package org.apache.cassandra.cql3;
+import com.google.common.base.Objects;
+
import org.apache.cassandra.db.marshal.AbstractType;
public class ColumnSpecification
@@ -40,4 +42,24 @@ public class ColumnSpecification
// 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/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 95e0441..1c19760 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -60,7 +60,10 @@ 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
{
@@ -184,6 +187,14 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return boundTerms;
}
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public Selection getSelection()
+ {
+ return selection;
+ }
+
public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
{
state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT);
@@ -580,7 +591,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return new SliceQueryFilter(slices, isReversed, limit, toGroup);
}
- private int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
{
int l = Integer.MAX_VALUE;
if (limit != null)
@@ -1067,6 +1081,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
variables);
}
+ /**
+ * May be used by custom QueryHandler implementations
+ */
public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
{
if (!usesSecondaryIndexing || restrictedNames.isEmpty())
@@ -1446,7 +1463,21 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return true;
}
- private boolean hasClusteringColumnsRestriction()
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public boolean hasPartitionKeyRestriction()
+ {
+ for (int i = 0; i < keyRestrictions.length; i++)
+ if (keyRestrictions[i] != null)
+ return true;
+ return false;
+ }
+
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public boolean hasClusteringColumnsRestriction()
{
for (int i = 0; i < columnRestrictions.length; i++)
if (columnRestrictions[i] != null)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index 223f698..50a34bf 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -18,8 +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 org.apache.cassandra.cql3.*;
import org.apache.cassandra.cql3.functions.Function;
@@ -37,14 +38,14 @@ import org.apache.cassandra.utils.ByteBufferUtil;
public abstract class Selection
{
private final List<CFDefinition.Name> columns;
- private final List<ColumnSpecification> metadata;
+ private final SelectionColumns columnMapping;
private final boolean collectTimestamps;
private final boolean collectTTLs;
- protected Selection(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
+ protected Selection(List<CFDefinition.Name> columns, SelectionColumns columnMapping, boolean collectTimestamps, boolean collectTTLs)
{
this.columns = columns;
- this.metadata = metadata;
+ this.columnMapping = columnMapping;
this.collectTimestamps = collectTimestamps;
this.collectTTLs = collectTTLs;
}
@@ -57,7 +58,7 @@ public abstract class Selection
public ResultSet.Metadata getResultMetadata()
{
- return new ResultSet.Metadata(metadata);
+ return new ResultSet.Metadata(columnMapping.getColumnSpecifications());
}
public static Selection wildcard(CFDefinition cfDef)
@@ -94,21 +95,28 @@ public abstract class Selection
return idx;
}
- private static Selector makeSelector(CFDefinition cfDef, RawSelector raw, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+ private static Selector makeSelector(CFDefinition cfDef,
+ RawSelector raw,
+ List<CFDefinition.Name> names,
+ SelectionColumnMapping columnMapping) throws InvalidRequestException
{
Selectable selectable = raw.selectable.prepare(cfDef.cfm);
- return makeSelector(cfDef, selectable, raw.alias, names, metadata);
+ return makeSelector(cfDef, selectable, raw.alias, names, columnMapping);
}
- private static Selector makeSelector(CFDefinition cfDef, Selectable selectable, ColumnIdentifier alias, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+ private static Selector makeSelector(CFDefinition cfDef,
+ Selectable selectable,
+ ColumnIdentifier alias,
+ List<CFDefinition.Name> names,
+ SelectionColumnMapping columnMapping) throws InvalidRequestException
{
if (selectable instanceof ColumnIdentifier)
{
- CFDefinition.Name name = cfDef.get((ColumnIdentifier)selectable);
+ CFDefinition.Name name = cfDef.get((ColumnIdentifier) selectable);
if (name == null)
throw new InvalidRequestException(String.format("Undefined name %s in selection clause", selectable));
- if (metadata != null)
- metadata.add(alias == null ? name : makeAliasSpec(cfDef, name.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);
}
else if (selectable instanceof Selectable.WritetimeOrTTL)
@@ -121,25 +129,26 @@ public abstract class Selection
throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", name));
if (name.type.isCollection())
throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl"));
-
- if (metadata != null)
- metadata.add(makeWritetimeOrTTLSpec(cfDef, tot, alias));
+ if (columnMapping != null)
+ columnMapping.addMapping(makeWritetimeOrTTLSpec(cfDef, tot, alias), name);
return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime);
}
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
+ SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
for (Selectable rawArg : withFun.args)
- args.add(makeSelector(cfDef, rawArg, null, names, null));
+ args.add(makeSelector(cfDef, rawArg, null, names, tmpMapping));
AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfDef.cfm.ksName, cfDef.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);
- if (metadata != null)
- metadata.add(spec);
+ if (columnMapping != null)
+ columnMapping.addMapping(spec, tmpMapping.getMappings().values());
return new FunctionSelector(fun, args);
}
}
@@ -178,23 +187,23 @@ public abstract class Selection
if (needsProcessing)
{
List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>();
- 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(cfDef, rawSelector, names, metadata);
+ Selector selector = makeSelector(cfDef, rawSelector, names, columnMapping);
selectors.add(selector);
collectTimestamps |= selector.usesTimestamps();
collectTTLs |= selector.usesTTLs();
}
- return new SelectionWithProcessing(names, metadata, selectors, collectTimestamps, collectTTLs);
+ return new SelectionWithProcessing(names, columnMapping, selectors, collectTimestamps, collectTTLs);
}
else
{
List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(rawSelectors.size());
- List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
+ SelectionColumnMapping columnMapping = SelectionColumnMapping.newMapping();
for (RawSelector rawSelector : rawSelectors)
{
assert rawSelector.selectable instanceof ColumnIdentifier.Raw;
@@ -203,9 +212,12 @@ public abstract class Selection
if (name == null)
throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
names.add(name);
- metadata.add(rawSelector.alias == null ? name : makeAliasSpec(cfDef, name.type, rawSelector.alias));
+ columnMapping.addMapping(rawSelector.alias == null ? name : makeAliasSpec(cfDef,
+ name.type,
+ rawSelector.alias),
+ name);
}
- return new SimpleSelection(names, metadata, false);
+ return new SimpleSelection(names, columnMapping, false);
}
}
@@ -233,6 +245,14 @@ public abstract class Selection
return columns;
}
+ /**
+ * @return the mappings between resultset columns and the underlying columns
+ */
+ public SelectionColumns getColumnMapping()
+ {
+ return columnMapping;
+ }
+
public ResultSetBuilder resultSetBuilder(long now)
{
return new ResultSetBuilder(now);
@@ -264,7 +284,7 @@ public abstract class Selection
private ResultSetBuilder(long now)
{
- this.resultSet = new ResultSet(metadata);
+ this.resultSet = new ResultSet(columnMapping.getColumnSpecifications());
this.timestamps = collectTimestamps ? new long[columns.size()] : null;
this.ttls = collectTTLs ? new int[columns.size()] : null;
this.now = now;
@@ -321,17 +341,17 @@ public abstract class Selection
public SimpleSelection(List<CFDefinition.Name> columns, boolean isWildcard)
{
- this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
+ this(columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
}
- public SimpleSelection(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean isWildcard)
+ public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping, boolean isWildcard)
{
/*
* In theory, even a simple selection could have multiple time the same column, so we
* could filter those duplicate out of columns. But since we're very unlikely to
* get much duplicate in practice, it's more efficient not to bother.
*/
- super(columns, metadata, false, false);
+ super(columns, columnMapping, false, false);
this.isWildcard = isWildcard;
}
@@ -351,9 +371,9 @@ public abstract class Selection
{
private final List<Selector> selectors;
- public SelectionWithProcessing(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
+ public SelectionWithProcessing(List<CFDefinition.Name> columns, SelectionColumns columnMapping, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
{
- super(columns, metadata, collectTimestamps, collectTTLs);
+ super(columns, columnMapping, collectTimestamps, collectTTLs);
this.selectors = selectors;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
new file mode 100644
index 0000000..d09612f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
@@ -0,0 +1,106 @@
+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.collect.*;
+
+import org.apache.cassandra.cql3.CFDefinition;
+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 SelectionColumnMapping()
+ {
+ this.columnMappings = LinkedHashMultimap.create();
+ }
+
+ protected static SelectionColumnMapping newMapping()
+ {
+ return new SelectionColumnMapping();
+ }
+
+ protected static SelectionColumnMapping simpleMapping(List<CFDefinition.Name> columnDefinitions)
+ {
+ SelectionColumnMapping mapping = new SelectionColumnMapping();
+ for (CFDefinition.Name def: columnDefinitions)
+ mapping.addMapping(def, def);
+ return mapping;
+ }
+
+ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, CFDefinition.Name column)
+ {
+ columnMappings.put(colSpec, column);
+ return this;
+ }
+
+ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<CFDefinition.Name> 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());
+ }
+
+ public Multimap<ColumnSpecification, CFDefinition.Name> getMappings()
+ {
+ return Multimaps.unmodifiableMultimap(columnMappings);
+ }
+
+ public boolean equals(Object obj)
+ {
+ if (obj == null)
+ return false;
+
+ if (!(obj instanceof SelectionColumns))
+ return false;
+
+ return Objects.equals(columnMappings, ((SelectionColumns) obj).getMappings());
+ }
+
+ public int hashCode()
+ {
+ return Objects.hashCode(columnMappings);
+ }
+
+ public String toString()
+ {
+ final Function<CFDefinition.Name, String> getDefName = new Function<CFDefinition.Name, String>()
+ {
+ public String apply(CFDefinition.Name name)
+ {
+ return 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)
+ {
+ 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/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
new file mode 100644
index 0000000..3053f99
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
@@ -0,0 +1,19 @@
+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.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();
+}
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
new file mode 100644
index 0000000..9c31653
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
@@ -0,0 +1,244 @@
+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.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
+{
+ 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));
+ }
+
+ @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)");
+ testSimpleTypes();
+ testWildcard();
+ testSimpleTypesWithAliases();
+ 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)))");
+ ColumnSpecification tokenSpec = columnSpecification("token(a, b)", BytesType.instance);
+ SelectionColumns 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"));
+
+ 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"));
+
+ 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"));
+
+ assertEquals(expected, extractColumnMappingFromSelect(
+ "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS k_alias," +
+ " writetime(v1)," +
+ " ttl(v2) as ttl_alias" +
+ " 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)
+ {
+ return Schema.instance.getCFMetaData(KEYSPACE, tableName)
+ .getCfDef()
+ .get(new ColumnIdentifier(name, true));
+
+ }
+
+ private Iterable<CFDefinition.Name> columnDefinitions(String...name)
+ {
+ List<CFDefinition.Name> 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();
+ }
+ }
+}
[12/12] cassandra git commit: Merge branch 'cassandra-2.2' into trunk
Posted by sa...@apache.org.
Merge branch 'cassandra-2.2' into trunk
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/40424ee5
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/40424ee5
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/40424ee5
Branch: refs/heads/trunk
Commit: 40424ee598d2c64e7530a5bf0febd0bd4e8d825c
Parents: 67baaef 1f8516d
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 18 17:44:34 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jun 18 17:44:34 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/cassandra/cql3/ColumnIdentifier.java | 6 +-
.../cassandra/cql3/ColumnSpecification.java | 6 +
.../restrictions/StatementRestrictions.java | 22 +-
.../selection/AbstractFunctionSelector.java | 13 +
.../cassandra/cql3/selection/FieldSelector.java | 8 +-
.../cassandra/cql3/selection/Selectable.java | 10 +-
.../cassandra/cql3/selection/Selection.java | 58 +--
.../cql3/selection/SelectionColumnMapping.java | 118 +++++++
.../cql3/selection/SelectionColumns.java | 18 +
.../cassandra/cql3/selection/Selector.java | 13 +
.../cql3/selection/SelectorFactories.java | 2 +-
.../cql3/selection/SimpleSelector.java | 17 +-
.../cql3/selection/WritetimeOrTTLSelector.java | 14 +-
.../cql3/statements/SelectStatement.java | 28 +-
.../selection/SelectionColumnMappingTest.java | 353 +++++++++++++++++++
16 files changed, 633 insertions(+), 54 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/40424ee5/CHANGES.txt
----------------------------------------------------------------------
[07/12] cassandra git commit: Merge branch 'cassandra-2.0' into
cassandra-2.1
Posted by sa...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/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/trunk
Commit: 0452e74f59182aee515cc6920e35287def86c9fe
Parents: 9966419 f32cff8
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 18 17:32:04 2015 +0100
Committer: Sam Tunnicliffe <sa...@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();
- }
- }
+ }
[08/12] cassandra git commit: Merge branch 'cassandra-2.1' into
cassandra-2.2
Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/Selector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selector.java
index 747dc60,0000000..9b7f0ba
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selector.java
@@@ -1,179 -1,0 +1,192 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+
+import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A <code>Selector</code> is used to convert the data returned by the storage engine into the data requested by the
+ * user. They correspond to the <selector> elements from the select clause.
+ * <p>Since the introduction of aggregation, <code>Selector</code>s cannot be called anymore by multiple threads
+ * as they have an internal state.</p>
+ */
+public abstract class Selector implements AssignmentTestable
+{
+ /**
+ * A factory for <code>Selector</code> instances.
+ */
+ public static abstract class Factory
+ {
+ public Iterable<Function> getFunctions()
+ {
+ return Collections.emptySet();
+ }
+
+ /**
+ * Returns the column specification corresponding to the output value of the selector instances created by
+ * this factory.
+ *
+ * @param cfm the column family meta data
+ * @return a column specification
+ */
+ public final ColumnSpecification getColumnSpecification(CFMetaData cfm)
+ {
+ return new ColumnSpecification(cfm.ksName,
+ cfm.cfName,
+ new ColumnIdentifier(getColumnName(), true),
+ getReturnType());
+ }
+
+ /**
+ * Creates a new <code>Selector</code> instance.
+ *
+ * @return a new <code>Selector</code> instance
+ */
+ public abstract Selector newInstance() throws InvalidRequestException;
+
+ /**
+ * Checks if this factory creates selectors instances that creates aggregates.
+ *
+ * @return <code>true</code> if this factory creates selectors instances that creates aggregates,
+ * <code>false</code> otherwise
+ */
+ public boolean isAggregateSelectorFactory()
+ {
+ return false;
+ }
+
+ /**
+ * Checks if this factory creates <code>writetime</code> selectors instances.
+ *
+ * @return <code>true</code> if this factory creates <code>writetime</code> selectors instances,
+ * <code>false</code> otherwise
+ */
+ public boolean isWritetimeSelectorFactory()
+ {
+ return false;
+ }
+
+ /**
+ * Checks if this factory creates <code>TTL</code> selectors instances.
+ *
+ * @return <code>true</code> if this factory creates <code>TTL</code> selectors instances,
+ * <code>false</code> otherwise
+ */
+ public boolean isTTLSelectorFactory()
+ {
+ return false;
+ }
+
+ /**
+ * Returns the name of the column corresponding to the output value of the selector instances created by
+ * this factory.
+ *
+ * @return a column name
+ */
+ protected abstract String getColumnName();
+
+ /**
+ * Returns the type of the values returned by the selector instances created by this factory.
+ *
+ * @return the selector output type
+ */
+ protected abstract AbstractType<?> getReturnType();
+
++ /**
++ * Record a mapping between the ColumnDefinitions that are used by the selector
++ * instances created by this factory and a column in the ResultSet.Metadata
++ * returned with a query. In most cases, this is likely to be a 1:1 mapping,
++ * but some selector instances may utilise multiple columns (or none at all)
++ * to produce a value (i.e. functions).
++ *
++ * @param mapping the instance of the column mapping belonging to the current query's Selection
++ * @param resultsColumn the column in the ResultSet.Metadata to which the ColumnDefinitions used
++ * by the Selector are to be mapped
++ */
++ protected abstract void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn);
+ }
+
+ /**
+ * Add the current value from the specified <code>ResultSetBuilder</code>.
+ *
+ * @param protocolVersion protocol version used for serialization
+ * @param rs the <code>ResultSetBuilder</code>
+ * @throws InvalidRequestException if a problem occurs while add the input value
+ */
+ public abstract void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException;
+
+ /**
+ * Returns the selector output.
+ *
+ * @param protocolVersion protocol version used for serialization
+ * @return the selector output
+ * @throws InvalidRequestException if a problem occurs while computing the output value
+ */
+ public abstract ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException;
+
+ /**
+ * Returns the <code>Selector</code> output type.
+ *
+ * @return the <code>Selector</code> output type.
+ */
+ public abstract AbstractType<?> getType();
+
+ /**
+ * Checks if this <code>Selector</code> is creating aggregates.
+ *
+ * @return <code>true</code> if this <code>Selector</code> is creating aggregates <code>false</code>
+ * otherwise.
+ */
+ public boolean isAggregate()
+ {
+ return false;
+ }
+
+ /**
+ * Reset the internal state of this <code>Selector</code>.
+ */
+ public abstract void reset();
+
+ public final AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+ {
+ // We should ignore the fact that the output type is frozen in our comparison as functions do not support
+ // frozen types for arguments
+ AbstractType<?> receiverType = receiver.type;
+ if (getType().isFrozenCollection())
+ receiverType = receiverType.freeze();
+
+ if (receiverType.equals(getType()))
+ return AssignmentTestable.TestResult.EXACT_MATCH;
+
+ if (receiverType.isValueCompatibleWith(getType()))
+ return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+ return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
index beb7399,0000000..81905e6
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
@@@ -1,206 -1,0 +1,206 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.util.*;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A set of <code>Selector</code> factories.
+ */
+final class SelectorFactories implements Iterable<Selector.Factory>
+{
+ /**
+ * The <code>Selector</code> factories.
+ */
+ private final List<Selector.Factory> factories;
+
+ /**
+ * <code>true</code> if one of the factory creates writetime selectors.
+ */
+ private boolean containsWritetimeFactory;
+
+ /**
+ * <code>true</code> if one of the factory creates TTL selectors.
+ */
+ private boolean containsTTLFactory;
+
+ /**
+ * The number of factories creating aggregates.
+ */
+ private int numberOfAggregateFactories;
+
+ /**
+ * Creates a new <code>SelectorFactories</code> instance and collect the column definitions.
+ *
+ * @param selectables the <code>Selectable</code>s for which the factories must be created
+ * @param cfm the Column Family Definition
+ * @param defs the collector parameter for the column definitions
+ * @return a new <code>SelectorFactories</code> instance
+ * @throws InvalidRequestException if a problem occurs while creating the factories
+ */
+ public static SelectorFactories createFactoriesAndCollectColumnDefinitions(List<Selectable> selectables,
+ CFMetaData cfm,
+ List<ColumnDefinition> defs)
+ throws InvalidRequestException
+ {
+ return new SelectorFactories(selectables, cfm, defs);
+ }
+
+ private SelectorFactories(List<Selectable> selectables,
+ CFMetaData cfm,
+ List<ColumnDefinition> defs)
+ throws InvalidRequestException
+ {
+ factories = new ArrayList<>(selectables.size());
+
+ for (Selectable selectable : selectables)
+ {
+ Factory factory = selectable.newSelectorFactory(cfm, defs);
+ containsWritetimeFactory |= factory.isWritetimeSelectorFactory();
+ containsTTLFactory |= factory.isTTLSelectorFactory();
+ if (factory.isAggregateSelectorFactory())
+ ++numberOfAggregateFactories;
+ factories.add(factory);
+ }
+ }
+
+ public Iterable<Function> getFunctions()
+ {
+ Iterable<Function> functions = Collections.emptySet();
+ for (Factory factory : factories)
+ if (factory != null)
+ functions = Iterables.concat(functions, factory.getFunctions());
+ return functions;
+ }
+
+ /**
+ * Adds a new <code>Selector.Factory</code> for a column that is needed only for ORDER BY purposes.
+ * @param def the column that is needed for ordering
+ * @param index the index of the column definition in the Selection's list of columns
+ */
+ public void addSelectorForOrdering(ColumnDefinition def, int index)
+ {
- factories.add(SimpleSelector.newFactory(def.name.toString(), index, def.type));
++ factories.add(SimpleSelector.newFactory(def, index));
+ }
+
+ /**
+ * Checks if this <code>SelectorFactories</code> contains only factories for aggregates.
+ *
+ * @return <code>true</code> if this <code>SelectorFactories</code> contains only factories for aggregates,
+ * <code>false</code> otherwise.
+ */
+ public boolean containsOnlyAggregateFunctions()
+ {
+ int size = factories.size();
+ return size != 0 && numberOfAggregateFactories == size;
+ }
+
+ /**
+ * Whether the selector built by this factory does aggregation or not (either directly or in a sub-selector).
+ *
+ * @return <code>true</code> if the selector built by this factor does aggregation, <code>false</code> otherwise.
+ */
+ public boolean doesAggregation()
+ {
+ return numberOfAggregateFactories > 0;
+ }
+
+ /**
+ * Checks if this <code>SelectorFactories</code> contains at least one factory for writetime selectors.
+ *
+ * @return <code>true</code> if this <code>SelectorFactories</code> contains at least one factory for writetime
+ * selectors, <code>false</code> otherwise.
+ */
+ public boolean containsWritetimeSelectorFactory()
+ {
+ return containsWritetimeFactory;
+ }
+
+ /**
+ * Checks if this <code>SelectorFactories</code> contains at least one factory for TTL selectors.
+ *
+ * @return <code>true</code> if this <code>SelectorFactories</code> contains at least one factory for TTL
+ * selectors, <code>false</code> otherwise.
+ */
+ public boolean containsTTLSelectorFactory()
+ {
+ return containsTTLFactory;
+ }
+
+ /**
+ * Creates a list of new <code>Selector</code> instances.
+ * @return a list of new <code>Selector</code> instances.
+ */
+ public List<Selector> newInstances() throws InvalidRequestException
+ {
+ List<Selector> selectors = new ArrayList<>(factories.size());
+ for (Selector.Factory factory : factories)
+ {
+ selectors.add(factory.newInstance());
+ }
+ return selectors;
+ }
+
+ public Iterator<Factory> iterator()
+ {
+ return factories.iterator();
+ }
+
+ /**
+ * Returns the names of the columns corresponding to the output values of the selector instances created by
+ * these factories.
+ *
+ * @return a list of column names
+ */
+ public List<String> getColumnNames()
+ {
+ return Lists.transform(factories, new com.google.common.base.Function<Selector.Factory, String>()
+ {
+ public String apply(Selector.Factory factory)
+ {
+ return factory.getColumnName();
+ }
+ });
+ }
+
+ /**
+ * Returns a list of the return types of the selector instances created by these factories.
+ *
+ * @return a list of types
+ */
+ public List<AbstractType<?>> getReturnTypes()
+ {
+ return Lists.transform(factories, new com.google.common.base.Function<Selector.Factory, AbstractType<?>>()
+ {
+ public AbstractType<?> apply(Selector.Factory factory)
+ {
+ return factory.getReturnType();
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
index c2edaed,0000000..6c4dc04
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
@@@ -1,93 -1,0 +1,100 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+
++import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public final class SimpleSelector extends Selector
+{
+ private final String columnName;
+ private final int idx;
+ private final AbstractType<?> type;
+ private ByteBuffer current;
+
- public static Factory newFactory(final String columnName, final int idx, final AbstractType<?> type)
++ public static Factory newFactory(final ColumnDefinition def, final int idx)
+ {
+ return new Factory()
+ {
+ @Override
+ protected String getColumnName()
+ {
- return columnName;
++ return def.name.toString();
+ }
+
+ @Override
+ protected AbstractType<?> getReturnType()
+ {
- return type;
++ return def.type;
++ }
++
++ protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultColumn)
++ {
++ mapping.addMapping(resultColumn, def);
+ }
+
+ @Override
+ public Selector newInstance()
+ {
- return new SimpleSelector(columnName, idx, type);
++ return new SimpleSelector(def.name.toString(), idx, def.type);
+ }
+ };
+ }
+
+ @Override
+ public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+ {
+ current = rs.current.get(idx);
+ }
+
+ @Override
+ public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+ {
+ return current;
+ }
+
+ @Override
+ public void reset()
+ {
+ current = null;
+ }
+
+ @Override
+ public AbstractType<?> getType()
+ {
+ return type;
+ }
+
+ @Override
+ public String toString()
+ {
+ return columnName;
+ }
+
+ private SimpleSelector(String columnName, int idx, AbstractType<?> type)
+ {
+ this.columnName = columnName;
+ this.idx = idx;
+ this.type = type;
+ }
- }
++}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
index a1ecd3d,0000000..b3607f3
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
@@@ -1,108 -1,0 +1,116 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
++import java.util.Collections;
+
++import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+final class WritetimeOrTTLSelector extends Selector
+{
+ private final String columnName;
+ private final int idx;
+ private final boolean isWritetime;
+ private ByteBuffer current;
+
- public static Factory newFactory(final String columnName, final int idx, final boolean isWritetime)
++ public static Factory newFactory(final ColumnDefinition def, final int idx, final boolean isWritetime)
+ {
+ return new Factory()
+ {
+ protected String getColumnName()
+ {
- return String.format("%s(%s)", isWritetime ? "writetime" : "ttl", columnName);
++ return String.format("%s(%s)", isWritetime ? "writetime" : "ttl", def.name.toString());
+ }
+
+ protected AbstractType<?> getReturnType()
+ {
+ return isWritetime ? LongType.instance : Int32Type.instance;
+ }
+
++ protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
++ {
++ mapping.addMapping(resultsColumn, def);
++ }
++
+ public Selector newInstance()
+ {
- return new WritetimeOrTTLSelector(columnName, idx, isWritetime);
++ return new WritetimeOrTTLSelector(def.name.toString(), idx, isWritetime);
+ }
+
+ public boolean isWritetimeSelectorFactory()
+ {
+ return isWritetime;
+ }
+
+ public boolean isTTLSelectorFactory()
+ {
+ return !isWritetime;
+ }
+ };
+ }
+
+ public void addInput(int protocolVersion, ResultSetBuilder rs)
+ {
+ if (isWritetime)
+ {
+ long ts = rs.timestamps[idx];
+ current = ts != Long.MIN_VALUE ? ByteBufferUtil.bytes(ts) : null;
+ }
+ else
+ {
+ int ttl = rs.ttls[idx];
+ current = ttl > 0 ? ByteBufferUtil.bytes(ttl) : null;
+ }
+ }
+
+ public ByteBuffer getOutput(int protocolVersion)
+ {
+ return current;
+ }
+
+ public void reset()
+ {
+ current = null;
+ }
+
+ public AbstractType<?> getType()
+ {
+ return isWritetime ? LongType.instance : Int32Type.instance;
+ }
+
+ @Override
+ public String toString()
+ {
+ return columnName;
+ }
+
+ private WritetimeOrTTLSelector(String columnName, int idx, boolean isWritetime)
+ {
+ this.columnName = columnName;
+ this.idx = idx;
+ this.isWritetime = isWritetime;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index dfb0d07,d0566eb..8ce555f
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -288,9 -341,11 +292,25 @@@ public class SelectStatement implement
return cfm.cfName;
}
++ /**
++ * May be used by custom QueryHandler implementations
++ */
++ public Selection getSelection()
++ {
++ return selection;
++ }
++
++ /**
++ * May be used by custom QueryHandler implementations
++ */
++ public StatementRestrictions getRestrictions()
++ {
++ return restrictions;
++ }
++
private List<ReadCommand> getSliceCommands(QueryOptions options, int limit, long now) throws RequestValidationException
{
- Collection<ByteBuffer> keys = getKeys(options);
- if (keys.isEmpty()) // in case of IN () for (the last column of) the partition key.
- return null;
+ Collection<ByteBuffer> keys = restrictions.getPartitionKeys(options);
List<ReadCommand> commands = new ArrayList<>(keys.size());
@@@ -453,14 -568,18 +473,17 @@@
return new SliceQueryFilter(slices, isReversed, limit, toGroup);
}
- private int getLimit(QueryOptions options) throws InvalidRequestException
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public int getLimit(QueryOptions options) throws InvalidRequestException
{
- int l = Integer.MAX_VALUE;
if (limit != null)
{
- ByteBuffer b = limit.bindAndGet(options);
- if (b == null)
- throw new InvalidRequestException("Invalid null value of limit");
-
+ ByteBuffer b = checkNotNull(limit.bindAndGet(options), "Invalid null value of limit");
+ // treat UNSET limit value as 'unlimited'
+ if (b == UNSET_BYTE_BUFFER)
+ return Integer.MAX_VALUE;
try
{
Int32Type.instance.validate(b);
@@@ -530,18 -853,300 +553,21 @@@
}
}
- /** Returns true if a non-frozen collection is selected, false otherwise. */
- private boolean selectACollection()
- {
- if (!cfm.comparator.hasCollections())
- return false;
-
- for (ColumnDefinition def : selection.getColumns())
- {
- if (def.type.isCollection() && def.type.isMultiCell())
- return true;
- }
-
- return false;
- }
-
- @VisibleForTesting
- static List<Composite> buildBound(Bound bound,
- List<ColumnDefinition> defs,
- Restriction[] restrictions,
- boolean isReversed,
- CType type,
- QueryOptions options) throws InvalidRequestException
- {
- CBuilder builder = type.builder();
-
- // 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),
- // it only depends on whether the slice is reversed
- Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
- for (int i = 0, m = defs.size(); i < m; i++)
- {
- ColumnDefinition def = defs.get(i);
-
- // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
- // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
- // But if the actual comparator itself is reversed, we must inversed the bounds too.
- Bound b = isReversed == isReversedType(def) ? bound : Bound.reverse(bound);
- Restriction r = restrictions[def.position()];
- if (isNullRestriction(r, b) || !r.canEvaluateWithSlices())
- {
- // 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.
- Composite prefix = builder.build();
- return Collections.singletonList(eocBound == Bound.END ? prefix.end() : prefix.start());
- }
- if (r.isSlice())
- {
- if (r.isMultiColumn())
- {
- MultiColumnRestriction.Slice slice = (MultiColumnRestriction.Slice) r;
-
- if (!slice.hasBound(b))
- {
- Composite prefix = builder.build();
- return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
- ? prefix.end()
- : prefix);
- }
-
- List<ByteBuffer> vals = slice.componentBounds(b, options);
-
- for (int j = 0, n = vals.size(); j < n; j++)
- addValue(builder, defs.get(i + j), vals.get(j)) ;
- }
- else
- {
- builder.add(getSliceValue(r, b, options));
- }
- Operator relType = ((Restriction.Slice)r).getRelation(eocBound, b);
- return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
- }
-
- if (r.isIN())
- {
- // 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> inValues = new TreeSet<>(isReversed ? type.reverseComparator() : type);
-
- if (r.isMultiColumn())
- {
- List<List<ByteBuffer>> splitInValues = ((MultiColumnRestriction.IN) r).splitValues(options);
-
- for (List<ByteBuffer> components : splitInValues)
- {
- for (int j = 0; j < components.size(); j++)
- if (components.get(j) == null)
- throw new InvalidRequestException("Invalid null value in condition for column " + defs.get(i + j).name);
-
- Composite prefix = builder.buildWith(components);
- inValues.add(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
- }
- return new ArrayList<>(inValues);
- }
-
- List<ByteBuffer> values = r.values(options);
- if (values.size() != 1)
- {
- // IN query, we only support it on the clustering columns
- assert def.position() == defs.size() - 1;
- for (ByteBuffer val : values)
- {
- if (val == null)
- throw new InvalidRequestException(String.format("Invalid null value in condition for column %s",
- def.name));
- Composite prefix = builder.buildWith(val);
- // See below for why this
- inValues.add(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
- }
- return new ArrayList<>(inValues);
- }
- }
-
- List<ByteBuffer> values = r.values(options);
-
- if (r.isMultiColumn())
- {
- for (int j = 0; j < values.size(); j++)
- addValue(builder, defs.get(i + j), values.get(j));
- i += values.size() - 1; // skips the processed columns
- }
- else
- {
- addValue(builder, def, values.get(0));
- }
- }
- // Means no relation at all or everything was an equal
- // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
- // 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
- // prefix.end() will sort after prefix (see #5240).
- Composite prefix = builder.build();
- return Collections.singletonList(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
- }
-
- /**
- * Adds an EOC to the specified Composite.
- *
- * @param composite the composite
- * @param eocBound the EOC bound
- * @return a new <code>Composite</code> with the EOC corresponding to the eocBound
- */
- private static Composite addEOC(Composite composite, Bound eocBound)
- {
- return eocBound == Bound.END ? composite.end() : composite.start();
- }
-
- /**
- * Adds the specified value to the specified builder
- *
- * @param builder the CBuilder to which the value must be added
- * @param def the column associated to the value
- * @param value the value to add
- * @throws InvalidRequestException if the value is null
- */
- private static void addValue(CBuilder builder, ColumnDefinition def, ByteBuffer value) throws InvalidRequestException
- {
- if (value == null)
- throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", def.name));
- builder.add(value);
- }
-
- private static Composite.EOC eocForRelation(Operator 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 static boolean isNullRestriction(Restriction r, Bound b)
- {
- return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
- }
-
- 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, options);
- if (val == null)
- throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
- return val;
- }
-
- private List<Composite> getRequestedBound(Bound b, QueryOptions options) throws InvalidRequestException
- {
- assert isColumnRange();
- return buildBound(b, cfm.clusteringColumns(), columnRestrictions, isReversed, cfm.comparator, options);
- }
-
+ /**
+ * May be used by custom QueryHandler implementations
+ */
public List<IndexExpression> getValidatedIndexExpressions(QueryOptions options) throws InvalidRequestException
{
- if (!usesSecondaryIndexing || restrictedColumns.isEmpty())
+ if (!restrictions.usesSecondaryIndexing())
return Collections.emptyList();
- List<IndexExpression> expressions = new ArrayList<IndexExpression>();
- for (ColumnDefinition def : restrictedColumns.keySet())
- {
- Restriction restriction;
- switch (def.kind)
- {
- case PARTITION_KEY:
- restriction = keyRestrictions[def.position()];
- break;
- case CLUSTERING_COLUMN:
- restriction = columnRestrictions[def.position()];
- break;
- case REGULAR:
- case STATIC:
- restriction = metadataRestrictions.get(def.name);
- break;
- default:
- // We don't allow restricting a COMPACT_VALUE for now in prepare.
- throw new AssertionError();
- }
-
- if (restriction.isSlice())
- {
- Restriction.Slice slice = (Restriction.Slice)restriction;
- for (Bound b : Bound.values())
- {
- if (slice.hasBound(b))
- {
- ByteBuffer value = validateIndexedValue(def, slice.bound(b, options));
- Operator op = slice.getIndexOperator(b);
- // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
- // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
- // use the underlying comparator as is.
- if (def.type instanceof ReversedType)
- op = reverse(op);
- expressions.add(new IndexExpression(def.name.bytes, op, value));
- }
- }
- }
- else if (restriction.isContains())
- {
- SingleColumnRestriction.Contains contains = (SingleColumnRestriction.Contains)restriction;
- for (ByteBuffer value : contains.values(options))
- {
- validateIndexedValue(def, value);
- expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS, value));
- }
- for (ByteBuffer key : contains.keys(options))
- {
- validateIndexedValue(def, key);
- expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS_KEY, key));
- }
- }
- else
- {
- ByteBuffer value;
- if (restriction.isMultiColumn())
- {
- List<ByteBuffer> values = restriction.values(options);
- value = values.get(def.position());
- }
- else
- {
- List<ByteBuffer> values = restriction.values(options);
- if (values.size() != 1)
- throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
+ SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
- value = values.get(0);
- }
+ List<IndexExpression> expressions = restrictions.getIndexExpressions(secondaryIndexManager, options);
- validateIndexedValue(def, value);
- expressions.add(new IndexExpression(def.name.bytes, Operator.EQ, value));
- }
- }
+ secondaryIndexManager.validateIndexSearchersForQuery(expressions);
- if (usesSecondaryIndexing)
- {
- ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
- SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
- secondaryIndexManager.validateIndexSearchersForQuery(expressions);
- }
-
return expressions;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
index 0000000,0000000..5bacf0d
new file mode 100644
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
@@@ -1,0 -1,0 +1,353 @@@
++package org.apache.cassandra.cql3.selection;
++
++import java.util.Collections;
++
++import com.google.common.collect.ImmutableList;
++import org.junit.Test;
++
++import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.Schema;
++import org.apache.cassandra.cql3.*;
++import org.apache.cassandra.cql3.statements.SelectStatement;
++import org.apache.cassandra.db.marshal.*;
++import org.apache.cassandra.exceptions.RequestValidationException;
++import org.apache.cassandra.service.ClientState;
++import org.apache.cassandra.utils.ByteBufferUtil;
++
++import static org.junit.Assert.assertEquals;
++import static org.junit.Assert.assertFalse;
++import static org.junit.Assert.assertTrue;
++
++public class SelectionColumnMappingTest extends CQLTester
++{
++ String tableName;
++ String typeName;
++ UserType userType;
++ String functionName;
++
++ @Test
++ public void testSelectionColumnMapping() throws Throwable
++ {
++ // Organised as a single test to avoid the overhead of
++ // table creation for each variant
++
++ 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 + ">)");
++ userType = Schema.instance.getKSMetaData(KEYSPACE).userTypes.getType(ByteBufferUtil.bytes(typeName));
++ functionName = createFunction(KEYSPACE, "int, ascii",
++ "CREATE FUNCTION %s (i int, a ascii) " +
++ "CALLED ON NULL INPUT " +
++ "RETURNS int " +
++ "LANGUAGE java " +
++ "AS 'return Integer.valueOf(i);'");
++ testSimpleTypes();
++ testWildcard();
++ testSimpleTypesWithAliases();
++ testUserTypes();
++ testUserTypesWithAliases();
++ testWritetimeAndTTL();
++ testWritetimeAndTTLWithAliases();
++ testFunction();
++ testNoArgFunction();
++ testUserDefinedFunction();
++ testOverloadedFunction();
++ testFunctionWithAlias();
++ testMultipleAliasesOnSameColumn();
++ testCount();
++ testMixedColumnTypes();
++ }
++
++ @Test
++ public void testMultipleArgumentFunction() throws Throwable
++ {
++ // demonstrate behaviour of token() with composite partition key
++ tableName = createTable("CREATE TABLE %s (a int, b text, PRIMARY KEY ((a, b)))");
++ ColumnSpecification tokenSpec = columnSpecification("system.token(a, b)", BytesType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(tokenSpec, columnDefinition("a"))
++ .addMapping(tokenSpec, columnDefinition("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
++ ColumnSpecification kSpec = columnSpecification("k", Int32Type.instance);
++ ColumnSpecification v1Spec = columnSpecification("v1", Int32Type.instance);
++ ColumnSpecification v2Spec = columnSpecification("v2", AsciiType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(kSpec, columnDefinition("k"))
++ .addMapping(v1Spec, columnDefinition("v1"))
++ .addMapping(v2Spec, columnDefinition("v2"));
++
++ assertEquals(expected, extractColumnMappingFromSelect("SELECT k, v1, v2 FROM %s"));
++ }
++
++ private void testWildcard() throws Throwable
++ {
++ // Wildcard select represents each column in the table with a ColumnDefinition
++ // in the ResultSet metadata
++ ColumnDefinition kSpec = columnDefinition("k");
++ ColumnDefinition v1Spec = columnDefinition("v1");
++ ColumnDefinition v2Spec = columnDefinition("v2");
++ ColumnDefinition v3Spec = columnDefinition("v3");
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(kSpec, columnDefinition("k"))
++ .addMapping(v1Spec, columnDefinition("v1"))
++ .addMapping(v2Spec, columnDefinition("v2"))
++ .addMapping(v3Spec, 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);
++ 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 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);
++ 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);
++ 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("system.intasblob(v1)", BytesType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(fnSpec, columnDefinition("v1"));
++
++ assertEquals(expected, extractColumnMappingFromSelect("SELECT intasblob(v1) FROM %s"));
++ }
++
++ private void testNoArgFunction() throws Throwable
++ {
++ // a no-arg function such as now() is represented in ResultSet.Metadata
++ // but has no mapping to any underlying column
++ ColumnSpecification fnSpec = columnSpecification("system.now()", TimeUUIDType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping().addMapping(fnSpec, null);
++
++ SelectionColumns actual = extractColumnMappingFromSelect("SELECT now() FROM %s");
++ assertEquals(expected, actual);
++ assertEquals(Collections.singletonList(fnSpec), actual.getColumnSpecifications());
++ assertTrue(actual.getMappings().isEmpty());
++ }
++
++ private void testOverloadedFunction() throws Throwable
++ {
++ String fnName = createFunction(KEYSPACE, "int",
++ "CREATE FUNCTION %s (input int) " +
++ "RETURNS NULL ON NULL INPUT " +
++ "RETURNS text " +
++ "LANGUAGE java " +
++ "AS 'return \"Hello World\";'");
++ createFunctionOverload(fnName, "text",
++ "CREATE FUNCTION %s (input text) " +
++ "RETURNS NULL ON NULL INPUT " +
++ "RETURNS text " +
++ "LANGUAGE java " +
++ "AS 'return \"Hello World\";'");
++
++ createFunctionOverload(fnName, "int, text",
++ "CREATE FUNCTION %s (input1 int, input2 text) " +
++ "RETURNS NULL ON NULL INPUT " +
++ "RETURNS text " +
++ "LANGUAGE java " +
++ "AS 'return \"Hello World\";'");
++ ColumnSpecification fnSpec1 = columnSpecification(fnName + "(v1)", UTF8Type.instance);
++ ColumnSpecification fnSpec2 = columnSpecification(fnName + "(v2)", UTF8Type.instance);
++ ColumnSpecification fnSpec3 = columnSpecification(fnName + "(v1, v2)", UTF8Type.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(fnSpec1, columnDefinition("v1"))
++ .addMapping(fnSpec2, columnDefinition("v2"))
++ .addMapping(fnSpec3, columnDefinition("v1"))
++ .addMapping(fnSpec3, columnDefinition("v2"));
++
++ String select = String.format("SELECT %1$s(v1), %1$s(v2), %1$s(v1, v2) FROM %%s", fnName);
++ SelectionColumns actual = extractColumnMappingFromSelect(select);
++
++ assertEquals(expected, actual);
++ assertEquals(ImmutableList.of(fnSpec1, fnSpec2, fnSpec3), actual.getColumnSpecifications());
++ }
++
++ private void testCount() throws Throwable
++ {
++ // SELECT COUNT does not necessarily include any mappings, but it must always return
++ // a singleton list from getColumnSpecifications() in order for the ResultSet.Metadata
++ // to be constructed correctly:
++ // * COUNT(*) / COUNT(1) do not generate any mappings, as no specific columns are referenced
++ // * COUNT(foo) does generate a mapping from the 'system.count' column spec to foo
++ ColumnSpecification count = columnSpecification("count", LongType.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(count, null);
++
++ SelectionColumns actual = extractColumnMappingFromSelect("SELECT COUNT(*) FROM %s");
++ assertEquals(expected, actual);
++ assertEquals(Collections.singletonList(count), actual.getColumnSpecifications());
++ assertTrue(actual.getMappings().isEmpty());
++
++ actual = extractColumnMappingFromSelect("SELECT COUNT(1) FROM %s");
++ assertEquals(expected, actual);
++ assertEquals(Collections.singletonList(count), actual.getColumnSpecifications());
++ assertTrue(actual.getMappings().isEmpty());
++
++ ColumnSpecification countV1 = columnSpecification("system.count(v1)", LongType.instance);
++ expected = SelectionColumnMapping.newMapping()
++ .addMapping(countV1, columnDefinition("v1"));
++ actual = extractColumnMappingFromSelect("SELECT COUNT(v1) FROM %s");
++ assertEquals(expected, actual);
++ assertEquals(Collections.singletonList(countV1), actual.getColumnSpecifications());
++ assertFalse(actual.getMappings().isEmpty());
++ }
++
++ private void testUserDefinedFunction() throws Throwable
++ {
++ // UDFs are basically represented in the same way as system functions
++ String functionCall = String.format("%s(v1, v2)", functionName);
++ ColumnSpecification fnSpec = columnSpecification(functionCall, Int32Type.instance);
++ SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
++ .addMapping(fnSpec, columnDefinition("v1"))
++ .addMapping(fnSpec, columnDefinition("v2"));
++
++ assertEquals(expected, extractColumnMappingFromSelect("SELECT " + functionCall + " 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);
++ 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);
++ 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);
++ ColumnSpecification f1Spec = columnSpecification("v3.f1", Int32Type.instance);
++ ColumnSpecification f2Spec = columnSpecification("f2_alias", UTF8Type.instance);
++ ColumnSpecification f3Spec = columnSpecification("v3", userType);
++
++ 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(f3Spec, columnDefinition("v3"));
++
++ assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS k_alias," +
++ " writetime(v1)," +
++ " 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 ColumnDefinition columnDefinition(String name)
++ {
++ return Schema.instance.getCFMetaData(KEYSPACE, tableName)
++ .getColumnDefinition(new ColumnIdentifier(name, true));
++
++ }
++
++ private ColumnSpecification columnSpecification(String name, AbstractType<?> type)
++ {
++ return new ColumnSpecification(KEYSPACE,
++ tableName,
++ new ColumnIdentifier(name, true),
++ type);
++ }
++}
[02/12] cassandra git commit: Expose some internals of
SelectStatement for inspection by QueryHandlers
Posted by sa...@apache.org.
Expose some internals of SelectStatement for inspection by QueryHandlers
patch by Sam Tunnicliffe; reviewed by Benjamin Lerer and Mick Semb Wever
for CASSANDRA-9532
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f32cff8e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f32cff8e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f32cff8e
Branch: refs/heads/cassandra-2.1
Commit: f32cff8e1fb69317219ffaee81b5861a54b83a1b
Parents: ad8047a
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 4 18:12:35 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jun 18 17:11:00 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../cassandra/cql3/ColumnSpecification.java | 22 ++
.../cql3/statements/SelectStatement.java | 37 ++-
.../cassandra/cql3/statements/Selection.java | 80 +++---
.../cql3/statements/SelectionColumnMapping.java | 106 ++++++++
.../cql3/statements/SelectionColumns.java | 19 ++
.../statements/SelectionColumnMappingTest.java | 244 +++++++++++++++++++
7 files changed, 476 insertions(+), 33 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 753fb1c..a235528 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.0.16:
+ * 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/f32cff8e/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
index 4dae701..089a1c5 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
@@ -17,6 +17,8 @@
*/
package org.apache.cassandra.cql3;
+import com.google.common.base.Objects;
+
import org.apache.cassandra.db.marshal.AbstractType;
public class ColumnSpecification
@@ -40,4 +42,24 @@ public class ColumnSpecification
// 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/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 95e0441..1c19760 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -60,7 +60,10 @@ 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
{
@@ -184,6 +187,14 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return boundTerms;
}
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public Selection getSelection()
+ {
+ return selection;
+ }
+
public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
{
state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT);
@@ -580,7 +591,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return new SliceQueryFilter(slices, isReversed, limit, toGroup);
}
- private int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
{
int l = Integer.MAX_VALUE;
if (limit != null)
@@ -1067,6 +1081,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
variables);
}
+ /**
+ * May be used by custom QueryHandler implementations
+ */
public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
{
if (!usesSecondaryIndexing || restrictedNames.isEmpty())
@@ -1446,7 +1463,21 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return true;
}
- private boolean hasClusteringColumnsRestriction()
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public boolean hasPartitionKeyRestriction()
+ {
+ for (int i = 0; i < keyRestrictions.length; i++)
+ if (keyRestrictions[i] != null)
+ return true;
+ return false;
+ }
+
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public boolean hasClusteringColumnsRestriction()
{
for (int i = 0; i < columnRestrictions.length; i++)
if (columnRestrictions[i] != null)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index 223f698..50a34bf 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -18,8 +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 org.apache.cassandra.cql3.*;
import org.apache.cassandra.cql3.functions.Function;
@@ -37,14 +38,14 @@ import org.apache.cassandra.utils.ByteBufferUtil;
public abstract class Selection
{
private final List<CFDefinition.Name> columns;
- private final List<ColumnSpecification> metadata;
+ private final SelectionColumns columnMapping;
private final boolean collectTimestamps;
private final boolean collectTTLs;
- protected Selection(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
+ protected Selection(List<CFDefinition.Name> columns, SelectionColumns columnMapping, boolean collectTimestamps, boolean collectTTLs)
{
this.columns = columns;
- this.metadata = metadata;
+ this.columnMapping = columnMapping;
this.collectTimestamps = collectTimestamps;
this.collectTTLs = collectTTLs;
}
@@ -57,7 +58,7 @@ public abstract class Selection
public ResultSet.Metadata getResultMetadata()
{
- return new ResultSet.Metadata(metadata);
+ return new ResultSet.Metadata(columnMapping.getColumnSpecifications());
}
public static Selection wildcard(CFDefinition cfDef)
@@ -94,21 +95,28 @@ public abstract class Selection
return idx;
}
- private static Selector makeSelector(CFDefinition cfDef, RawSelector raw, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+ private static Selector makeSelector(CFDefinition cfDef,
+ RawSelector raw,
+ List<CFDefinition.Name> names,
+ SelectionColumnMapping columnMapping) throws InvalidRequestException
{
Selectable selectable = raw.selectable.prepare(cfDef.cfm);
- return makeSelector(cfDef, selectable, raw.alias, names, metadata);
+ return makeSelector(cfDef, selectable, raw.alias, names, columnMapping);
}
- private static Selector makeSelector(CFDefinition cfDef, Selectable selectable, ColumnIdentifier alias, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+ private static Selector makeSelector(CFDefinition cfDef,
+ Selectable selectable,
+ ColumnIdentifier alias,
+ List<CFDefinition.Name> names,
+ SelectionColumnMapping columnMapping) throws InvalidRequestException
{
if (selectable instanceof ColumnIdentifier)
{
- CFDefinition.Name name = cfDef.get((ColumnIdentifier)selectable);
+ CFDefinition.Name name = cfDef.get((ColumnIdentifier) selectable);
if (name == null)
throw new InvalidRequestException(String.format("Undefined name %s in selection clause", selectable));
- if (metadata != null)
- metadata.add(alias == null ? name : makeAliasSpec(cfDef, name.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);
}
else if (selectable instanceof Selectable.WritetimeOrTTL)
@@ -121,25 +129,26 @@ public abstract class Selection
throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", name));
if (name.type.isCollection())
throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl"));
-
- if (metadata != null)
- metadata.add(makeWritetimeOrTTLSpec(cfDef, tot, alias));
+ if (columnMapping != null)
+ columnMapping.addMapping(makeWritetimeOrTTLSpec(cfDef, tot, alias), name);
return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime);
}
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
+ SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
for (Selectable rawArg : withFun.args)
- args.add(makeSelector(cfDef, rawArg, null, names, null));
+ args.add(makeSelector(cfDef, rawArg, null, names, tmpMapping));
AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfDef.cfm.ksName, cfDef.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);
- if (metadata != null)
- metadata.add(spec);
+ if (columnMapping != null)
+ columnMapping.addMapping(spec, tmpMapping.getMappings().values());
return new FunctionSelector(fun, args);
}
}
@@ -178,23 +187,23 @@ public abstract class Selection
if (needsProcessing)
{
List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>();
- 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(cfDef, rawSelector, names, metadata);
+ Selector selector = makeSelector(cfDef, rawSelector, names, columnMapping);
selectors.add(selector);
collectTimestamps |= selector.usesTimestamps();
collectTTLs |= selector.usesTTLs();
}
- return new SelectionWithProcessing(names, metadata, selectors, collectTimestamps, collectTTLs);
+ return new SelectionWithProcessing(names, columnMapping, selectors, collectTimestamps, collectTTLs);
}
else
{
List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(rawSelectors.size());
- List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
+ SelectionColumnMapping columnMapping = SelectionColumnMapping.newMapping();
for (RawSelector rawSelector : rawSelectors)
{
assert rawSelector.selectable instanceof ColumnIdentifier.Raw;
@@ -203,9 +212,12 @@ public abstract class Selection
if (name == null)
throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
names.add(name);
- metadata.add(rawSelector.alias == null ? name : makeAliasSpec(cfDef, name.type, rawSelector.alias));
+ columnMapping.addMapping(rawSelector.alias == null ? name : makeAliasSpec(cfDef,
+ name.type,
+ rawSelector.alias),
+ name);
}
- return new SimpleSelection(names, metadata, false);
+ return new SimpleSelection(names, columnMapping, false);
}
}
@@ -233,6 +245,14 @@ public abstract class Selection
return columns;
}
+ /**
+ * @return the mappings between resultset columns and the underlying columns
+ */
+ public SelectionColumns getColumnMapping()
+ {
+ return columnMapping;
+ }
+
public ResultSetBuilder resultSetBuilder(long now)
{
return new ResultSetBuilder(now);
@@ -264,7 +284,7 @@ public abstract class Selection
private ResultSetBuilder(long now)
{
- this.resultSet = new ResultSet(metadata);
+ this.resultSet = new ResultSet(columnMapping.getColumnSpecifications());
this.timestamps = collectTimestamps ? new long[columns.size()] : null;
this.ttls = collectTTLs ? new int[columns.size()] : null;
this.now = now;
@@ -321,17 +341,17 @@ public abstract class Selection
public SimpleSelection(List<CFDefinition.Name> columns, boolean isWildcard)
{
- this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
+ this(columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
}
- public SimpleSelection(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean isWildcard)
+ public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping, boolean isWildcard)
{
/*
* In theory, even a simple selection could have multiple time the same column, so we
* could filter those duplicate out of columns. But since we're very unlikely to
* get much duplicate in practice, it's more efficient not to bother.
*/
- super(columns, metadata, false, false);
+ super(columns, columnMapping, false, false);
this.isWildcard = isWildcard;
}
@@ -351,9 +371,9 @@ public abstract class Selection
{
private final List<Selector> selectors;
- public SelectionWithProcessing(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
+ public SelectionWithProcessing(List<CFDefinition.Name> columns, SelectionColumns columnMapping, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
{
- super(columns, metadata, collectTimestamps, collectTTLs);
+ super(columns, columnMapping, collectTimestamps, collectTTLs);
this.selectors = selectors;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
new file mode 100644
index 0000000..d09612f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
@@ -0,0 +1,106 @@
+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.collect.*;
+
+import org.apache.cassandra.cql3.CFDefinition;
+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 SelectionColumnMapping()
+ {
+ this.columnMappings = LinkedHashMultimap.create();
+ }
+
+ protected static SelectionColumnMapping newMapping()
+ {
+ return new SelectionColumnMapping();
+ }
+
+ protected static SelectionColumnMapping simpleMapping(List<CFDefinition.Name> columnDefinitions)
+ {
+ SelectionColumnMapping mapping = new SelectionColumnMapping();
+ for (CFDefinition.Name def: columnDefinitions)
+ mapping.addMapping(def, def);
+ return mapping;
+ }
+
+ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, CFDefinition.Name column)
+ {
+ columnMappings.put(colSpec, column);
+ return this;
+ }
+
+ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<CFDefinition.Name> 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());
+ }
+
+ public Multimap<ColumnSpecification, CFDefinition.Name> getMappings()
+ {
+ return Multimaps.unmodifiableMultimap(columnMappings);
+ }
+
+ public boolean equals(Object obj)
+ {
+ if (obj == null)
+ return false;
+
+ if (!(obj instanceof SelectionColumns))
+ return false;
+
+ return Objects.equals(columnMappings, ((SelectionColumns) obj).getMappings());
+ }
+
+ public int hashCode()
+ {
+ return Objects.hashCode(columnMappings);
+ }
+
+ public String toString()
+ {
+ final Function<CFDefinition.Name, String> getDefName = new Function<CFDefinition.Name, String>()
+ {
+ public String apply(CFDefinition.Name name)
+ {
+ return 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)
+ {
+ 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/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
new file mode 100644
index 0000000..3053f99
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
@@ -0,0 +1,19 @@
+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.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();
+}
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
new file mode 100644
index 0000000..9c31653
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
@@ -0,0 +1,244 @@
+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.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
+{
+ 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));
+ }
+
+ @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)");
+ testSimpleTypes();
+ testWildcard();
+ testSimpleTypesWithAliases();
+ 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)))");
+ ColumnSpecification tokenSpec = columnSpecification("token(a, b)", BytesType.instance);
+ SelectionColumns 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"));
+
+ 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"));
+
+ 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"));
+
+ assertEquals(expected, extractColumnMappingFromSelect(
+ "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS k_alias," +
+ " writetime(v1)," +
+ " ttl(v2) as ttl_alias" +
+ " 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)
+ {
+ return Schema.instance.getCFMetaData(KEYSPACE, tableName)
+ .getCfDef()
+ .get(new ColumnIdentifier(name, true));
+
+ }
+
+ private Iterable<CFDefinition.Name> columnDefinitions(String...name)
+ {
+ List<CFDefinition.Name> 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();
+ }
+ }
+}
[03/12] cassandra git commit: Expose some internals of
SelectStatement for inspection by QueryHandlers
Posted by sa...@apache.org.
Expose some internals of SelectStatement for inspection by QueryHandlers
patch by Sam Tunnicliffe; reviewed by Benjamin Lerer and Mick Semb Wever
for CASSANDRA-9532
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f32cff8e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f32cff8e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f32cff8e
Branch: refs/heads/cassandra-2.2
Commit: f32cff8e1fb69317219ffaee81b5861a54b83a1b
Parents: ad8047a
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 4 18:12:35 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jun 18 17:11:00 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../cassandra/cql3/ColumnSpecification.java | 22 ++
.../cql3/statements/SelectStatement.java | 37 ++-
.../cassandra/cql3/statements/Selection.java | 80 +++---
.../cql3/statements/SelectionColumnMapping.java | 106 ++++++++
.../cql3/statements/SelectionColumns.java | 19 ++
.../statements/SelectionColumnMappingTest.java | 244 +++++++++++++++++++
7 files changed, 476 insertions(+), 33 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 753fb1c..a235528 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.0.16:
+ * 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/f32cff8e/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
index 4dae701..089a1c5 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
@@ -17,6 +17,8 @@
*/
package org.apache.cassandra.cql3;
+import com.google.common.base.Objects;
+
import org.apache.cassandra.db.marshal.AbstractType;
public class ColumnSpecification
@@ -40,4 +42,24 @@ public class ColumnSpecification
// 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/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 95e0441..1c19760 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -60,7 +60,10 @@ 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
{
@@ -184,6 +187,14 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return boundTerms;
}
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public Selection getSelection()
+ {
+ return selection;
+ }
+
public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
{
state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT);
@@ -580,7 +591,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return new SliceQueryFilter(slices, isReversed, limit, toGroup);
}
- private int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
{
int l = Integer.MAX_VALUE;
if (limit != null)
@@ -1067,6 +1081,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
variables);
}
+ /**
+ * May be used by custom QueryHandler implementations
+ */
public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
{
if (!usesSecondaryIndexing || restrictedNames.isEmpty())
@@ -1446,7 +1463,21 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return true;
}
- private boolean hasClusteringColumnsRestriction()
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public boolean hasPartitionKeyRestriction()
+ {
+ for (int i = 0; i < keyRestrictions.length; i++)
+ if (keyRestrictions[i] != null)
+ return true;
+ return false;
+ }
+
+ /**
+ * May be used by custom QueryHandler implementations
+ */
+ public boolean hasClusteringColumnsRestriction()
{
for (int i = 0; i < columnRestrictions.length; i++)
if (columnRestrictions[i] != null)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index 223f698..50a34bf 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -18,8 +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 org.apache.cassandra.cql3.*;
import org.apache.cassandra.cql3.functions.Function;
@@ -37,14 +38,14 @@ import org.apache.cassandra.utils.ByteBufferUtil;
public abstract class Selection
{
private final List<CFDefinition.Name> columns;
- private final List<ColumnSpecification> metadata;
+ private final SelectionColumns columnMapping;
private final boolean collectTimestamps;
private final boolean collectTTLs;
- protected Selection(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
+ protected Selection(List<CFDefinition.Name> columns, SelectionColumns columnMapping, boolean collectTimestamps, boolean collectTTLs)
{
this.columns = columns;
- this.metadata = metadata;
+ this.columnMapping = columnMapping;
this.collectTimestamps = collectTimestamps;
this.collectTTLs = collectTTLs;
}
@@ -57,7 +58,7 @@ public abstract class Selection
public ResultSet.Metadata getResultMetadata()
{
- return new ResultSet.Metadata(metadata);
+ return new ResultSet.Metadata(columnMapping.getColumnSpecifications());
}
public static Selection wildcard(CFDefinition cfDef)
@@ -94,21 +95,28 @@ public abstract class Selection
return idx;
}
- private static Selector makeSelector(CFDefinition cfDef, RawSelector raw, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+ private static Selector makeSelector(CFDefinition cfDef,
+ RawSelector raw,
+ List<CFDefinition.Name> names,
+ SelectionColumnMapping columnMapping) throws InvalidRequestException
{
Selectable selectable = raw.selectable.prepare(cfDef.cfm);
- return makeSelector(cfDef, selectable, raw.alias, names, metadata);
+ return makeSelector(cfDef, selectable, raw.alias, names, columnMapping);
}
- private static Selector makeSelector(CFDefinition cfDef, Selectable selectable, ColumnIdentifier alias, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+ private static Selector makeSelector(CFDefinition cfDef,
+ Selectable selectable,
+ ColumnIdentifier alias,
+ List<CFDefinition.Name> names,
+ SelectionColumnMapping columnMapping) throws InvalidRequestException
{
if (selectable instanceof ColumnIdentifier)
{
- CFDefinition.Name name = cfDef.get((ColumnIdentifier)selectable);
+ CFDefinition.Name name = cfDef.get((ColumnIdentifier) selectable);
if (name == null)
throw new InvalidRequestException(String.format("Undefined name %s in selection clause", selectable));
- if (metadata != null)
- metadata.add(alias == null ? name : makeAliasSpec(cfDef, name.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);
}
else if (selectable instanceof Selectable.WritetimeOrTTL)
@@ -121,25 +129,26 @@ public abstract class Selection
throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", name));
if (name.type.isCollection())
throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl"));
-
- if (metadata != null)
- metadata.add(makeWritetimeOrTTLSpec(cfDef, tot, alias));
+ if (columnMapping != null)
+ columnMapping.addMapping(makeWritetimeOrTTLSpec(cfDef, tot, alias), name);
return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime);
}
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
+ SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
for (Selectable rawArg : withFun.args)
- args.add(makeSelector(cfDef, rawArg, null, names, null));
+ args.add(makeSelector(cfDef, rawArg, null, names, tmpMapping));
AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfDef.cfm.ksName, cfDef.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);
- if (metadata != null)
- metadata.add(spec);
+ if (columnMapping != null)
+ columnMapping.addMapping(spec, tmpMapping.getMappings().values());
return new FunctionSelector(fun, args);
}
}
@@ -178,23 +187,23 @@ public abstract class Selection
if (needsProcessing)
{
List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>();
- 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(cfDef, rawSelector, names, metadata);
+ Selector selector = makeSelector(cfDef, rawSelector, names, columnMapping);
selectors.add(selector);
collectTimestamps |= selector.usesTimestamps();
collectTTLs |= selector.usesTTLs();
}
- return new SelectionWithProcessing(names, metadata, selectors, collectTimestamps, collectTTLs);
+ return new SelectionWithProcessing(names, columnMapping, selectors, collectTimestamps, collectTTLs);
}
else
{
List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(rawSelectors.size());
- List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
+ SelectionColumnMapping columnMapping = SelectionColumnMapping.newMapping();
for (RawSelector rawSelector : rawSelectors)
{
assert rawSelector.selectable instanceof ColumnIdentifier.Raw;
@@ -203,9 +212,12 @@ public abstract class Selection
if (name == null)
throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
names.add(name);
- metadata.add(rawSelector.alias == null ? name : makeAliasSpec(cfDef, name.type, rawSelector.alias));
+ columnMapping.addMapping(rawSelector.alias == null ? name : makeAliasSpec(cfDef,
+ name.type,
+ rawSelector.alias),
+ name);
}
- return new SimpleSelection(names, metadata, false);
+ return new SimpleSelection(names, columnMapping, false);
}
}
@@ -233,6 +245,14 @@ public abstract class Selection
return columns;
}
+ /**
+ * @return the mappings between resultset columns and the underlying columns
+ */
+ public SelectionColumns getColumnMapping()
+ {
+ return columnMapping;
+ }
+
public ResultSetBuilder resultSetBuilder(long now)
{
return new ResultSetBuilder(now);
@@ -264,7 +284,7 @@ public abstract class Selection
private ResultSetBuilder(long now)
{
- this.resultSet = new ResultSet(metadata);
+ this.resultSet = new ResultSet(columnMapping.getColumnSpecifications());
this.timestamps = collectTimestamps ? new long[columns.size()] : null;
this.ttls = collectTTLs ? new int[columns.size()] : null;
this.now = now;
@@ -321,17 +341,17 @@ public abstract class Selection
public SimpleSelection(List<CFDefinition.Name> columns, boolean isWildcard)
{
- this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
+ this(columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
}
- public SimpleSelection(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean isWildcard)
+ public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping, boolean isWildcard)
{
/*
* In theory, even a simple selection could have multiple time the same column, so we
* could filter those duplicate out of columns. But since we're very unlikely to
* get much duplicate in practice, it's more efficient not to bother.
*/
- super(columns, metadata, false, false);
+ super(columns, columnMapping, false, false);
this.isWildcard = isWildcard;
}
@@ -351,9 +371,9 @@ public abstract class Selection
{
private final List<Selector> selectors;
- public SelectionWithProcessing(List<CFDefinition.Name> columns, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
+ public SelectionWithProcessing(List<CFDefinition.Name> columns, SelectionColumns columnMapping, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
{
- super(columns, metadata, collectTimestamps, collectTTLs);
+ super(columns, columnMapping, collectTimestamps, collectTTLs);
this.selectors = selectors;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
new file mode 100644
index 0000000..d09612f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
@@ -0,0 +1,106 @@
+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.collect.*;
+
+import org.apache.cassandra.cql3.CFDefinition;
+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 SelectionColumnMapping()
+ {
+ this.columnMappings = LinkedHashMultimap.create();
+ }
+
+ protected static SelectionColumnMapping newMapping()
+ {
+ return new SelectionColumnMapping();
+ }
+
+ protected static SelectionColumnMapping simpleMapping(List<CFDefinition.Name> columnDefinitions)
+ {
+ SelectionColumnMapping mapping = new SelectionColumnMapping();
+ for (CFDefinition.Name def: columnDefinitions)
+ mapping.addMapping(def, def);
+ return mapping;
+ }
+
+ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, CFDefinition.Name column)
+ {
+ columnMappings.put(colSpec, column);
+ return this;
+ }
+
+ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<CFDefinition.Name> 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());
+ }
+
+ public Multimap<ColumnSpecification, CFDefinition.Name> getMappings()
+ {
+ return Multimaps.unmodifiableMultimap(columnMappings);
+ }
+
+ public boolean equals(Object obj)
+ {
+ if (obj == null)
+ return false;
+
+ if (!(obj instanceof SelectionColumns))
+ return false;
+
+ return Objects.equals(columnMappings, ((SelectionColumns) obj).getMappings());
+ }
+
+ public int hashCode()
+ {
+ return Objects.hashCode(columnMappings);
+ }
+
+ public String toString()
+ {
+ final Function<CFDefinition.Name, String> getDefName = new Function<CFDefinition.Name, String>()
+ {
+ public String apply(CFDefinition.Name name)
+ {
+ return 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)
+ {
+ 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/f32cff8e/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
new file mode 100644
index 0000000..3053f99
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
@@ -0,0 +1,19 @@
+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.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();
+}
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f32cff8e/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
new file mode 100644
index 0000000..9c31653
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
@@ -0,0 +1,244 @@
+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.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
+{
+ 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));
+ }
+
+ @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)");
+ testSimpleTypes();
+ testWildcard();
+ testSimpleTypesWithAliases();
+ 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)))");
+ ColumnSpecification tokenSpec = columnSpecification("token(a, b)", BytesType.instance);
+ SelectionColumns 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"));
+
+ 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"));
+
+ 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"));
+
+ assertEquals(expected, extractColumnMappingFromSelect(
+ "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ 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"));
+
+ assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS k_alias," +
+ " writetime(v1)," +
+ " ttl(v2) as ttl_alias" +
+ " 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)
+ {
+ return Schema.instance.getCFMetaData(KEYSPACE, tableName)
+ .getCfDef()
+ .get(new ColumnIdentifier(name, true));
+
+ }
+
+ private Iterable<CFDefinition.Name> columnDefinitions(String...name)
+ {
+ List<CFDefinition.Name> 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();
+ }
+ }
+}
[09/12] cassandra git commit: Merge branch 'cassandra-2.1' into
cassandra-2.2
Posted by sa...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1f8516da
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1f8516da
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1f8516da
Branch: refs/heads/cassandra-2.2
Commit: 1f8516da82fc19500d3b00d37780265c34a358b0
Parents: a133992 0452e74
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 18 17:39:43 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jun 18 17:41:06 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/cassandra/cql3/ColumnIdentifier.java | 6 +-
.../cassandra/cql3/ColumnSpecification.java | 6 +
.../restrictions/StatementRestrictions.java | 22 +-
.../selection/AbstractFunctionSelector.java | 13 +
.../cassandra/cql3/selection/FieldSelector.java | 8 +-
.../cassandra/cql3/selection/Selectable.java | 10 +-
.../cassandra/cql3/selection/Selection.java | 58 +--
.../cql3/selection/SelectionColumnMapping.java | 118 +++++++
.../cql3/selection/SelectionColumns.java | 18 +
.../cassandra/cql3/selection/Selector.java | 13 +
.../cql3/selection/SelectorFactories.java | 2 +-
.../cql3/selection/SimpleSelector.java | 17 +-
.../cql3/selection/WritetimeOrTTLSelector.java | 14 +-
.../cql3/statements/SelectStatement.java | 28 +-
.../selection/SelectionColumnMappingTest.java | 353 +++++++++++++++++++
16 files changed, 633 insertions(+), 54 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 76a42ab,899ea7c..fb44c9a
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,16 -1,7 +1,17 @@@
-2.1.7
+2.2
+ * Rename class for DATE type in Java driver (CASSANDRA-9563)
+ * Duplicate compilation of UDFs on coordinator (CASSANDRA-9475)
+ * Fix connection leak in CqlRecordWriter (CASSANDRA-9576)
+ * Mlockall before opening system sstables & remove boot_without_jna option (CASSANDRA-9573)
+ * Add functions to convert timeuuid to date or time, deprecate dateOf and unixTimestampOf (CASSANDRA-9229)
+ * Make sure we cancel non-compacting sstables from LifecycleTransaction (CASSANDRA-9566)
+ * Fix deprecated repair JMX API (CASSANDRA-9570)
+ * Add logback metrics (CASSANDRA-9378)
+Merged from 2.1:
* Fix memory leak in Ref due to ConcurrentLinkedQueue.remove() behaviour (CASSANDRA-9549)
+ * Make rebuild only run one at a time (CASSANDRA-9119)
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/1f8516da/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 467b214,1501479..823af94
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@@ -17,20 -17,16 +17,20 @@@
*/
package org.apache.cassandra.cql3;
-import java.util.Locale;
+ import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Locale;
- import java.nio.ByteBuffer;
import org.apache.cassandra.cache.IMeasurableMemory;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.statements.Selectable;
+import org.apache.cassandra.cql3.selection.Selectable;
+import org.apache.cassandra.cql3.selection.Selector;
+import org.apache.cassandra.cql3.selection.SimpleSelector;
import org.apache.cassandra.db.marshal.AbstractType;
- import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.db.marshal.CompositeType;
import org.apache.cassandra.db.marshal.UTF8Type;
++import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.ObjectSizes;
import org.apache.cassandra.utils.memory.AbstractAllocator;
@@@ -109,14 -105,6 +109,14 @@@ public class ColumnIdentifier extends o
return new ColumnIdentifier(allocator.clone(bytes), text);
}
+ public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs) throws InvalidRequestException
+ {
+ ColumnDefinition def = cfm.getColumnDefinition(this);
+ if (def == null)
+ throw new InvalidRequestException(String.format("Undefined name %s in selection clause", this));
+
- return SimpleSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs), def.type);
++ return SimpleSelector.newFactory(def, addAndGetIndex(def, defs));
+ }
/**
* Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnSpecification.java
index bc5a914,f5f921d..e12a57e
--- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
@@@ -17,11 -17,9 +17,12 @@@
*/
package org.apache.cassandra.cql3;
+ import com.google.common.base.Objects;
-
import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+
+import java.util.Collection;
+import java.util.Iterator;
public class ColumnSpecification
{
@@@ -38,51 -36,23 +39,56 @@@
this.type = type;
}
- public boolean equals(Object obj)
+ /**
+ * Returns a new <code>ColumnSpecification</code> for the same column but with the specified alias.
+ *
+ * @param alias the column alias
+ * @return a new <code>ColumnSpecification</code> for the same column but with the specified alias.
+ */
+ public ColumnSpecification withAlias(ColumnIdentifier alias)
+ {
+ return new ColumnSpecification(ksName, cfName, alias, type);
+ }
+
+ public boolean isReversedType()
{
- if (null == obj)
+ return type instanceof ReversedType;
+ }
+
+ /**
+ * Returns true if all ColumnSpecifications are in the same table, false otherwise.
+ */
+ public static boolean allInSameTable(Collection<ColumnSpecification> names)
+ {
+ if (names == null || names.isEmpty())
return false;
- if(!(obj instanceof ColumnSpecification))
+ Iterator<ColumnSpecification> iter = names.iterator();
+ ColumnSpecification first = iter.next();
+ while (iter.hasNext())
+ {
+ ColumnSpecification name = iter.next();
+ if (!name.ksName.equals(first.ksName) || !name.cfName.equals(first.cfName))
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public boolean equals(Object other)
+ {
+ if (!(other 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);
+ ColumnSpecification that = (ColumnSpecification) other;
+ return this.ksName.equals(that.ksName) &&
+ this.cfName.equals(that.cfName) &&
+ this.name.equals(that.name) &&
+ this.type.equals(that.type);
}
+
+ public int hashCode()
+ {
+ return Objects.hashCode(ksName, cfName, name, type);
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index f848e2e,0000000..c10a56a
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@@ -1,572 -1,0 +1,572 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * The restrictions corresponding to the relations specified on the where-clause of CQL query.
+ */
+public final class StatementRestrictions
+{
+ /**
+ * The Column Family meta data
+ */
+ public final CFMetaData cfm;
+
+ /**
+ * Restrictions on partitioning columns
+ */
+ private PrimaryKeyRestrictions partitionKeyRestrictions;
+
+ /**
+ * Restrictions on clustering columns
+ */
+ private PrimaryKeyRestrictions clusteringColumnsRestrictions;
+
+ /**
+ * Restriction on non-primary key columns (i.e. secondary index restrictions)
+ */
+ private RestrictionSet nonPrimaryKeyRestrictions;
+
+ /**
+ * The restrictions used to build the index expressions
+ */
+ private final List<Restrictions> indexRestrictions = new ArrayList<>();
+
+ /**
+ * <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise
+ */
+ private boolean usesSecondaryIndexing;
+
+ /**
+ * Specify if the query will return a range of partition keys.
+ */
+ private boolean isKeyRange;
+
+ /**
+ * Creates a new empty <code>StatementRestrictions</code>.
+ *
+ * @param cfm the column family meta data
+ * @return a new empty <code>StatementRestrictions</code>.
+ */
+ public static StatementRestrictions empty(CFMetaData cfm)
+ {
+ return new StatementRestrictions(cfm);
+ }
+
+ private StatementRestrictions(CFMetaData cfm)
+ {
+ this.cfm = cfm;
+ this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
+ this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
+ this.nonPrimaryKeyRestrictions = new RestrictionSet();
+ }
+
+ public StatementRestrictions(CFMetaData cfm,
+ List<Relation> whereClause,
+ VariableSpecifications boundNames,
+ boolean selectsOnlyStaticColumns,
+ boolean selectACollection) throws InvalidRequestException
+ {
+ this.cfm = cfm;
+ this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
+ this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
+ this.nonPrimaryKeyRestrictions = new RestrictionSet();
+
+ /*
+ * WHERE clause. For a given entity, rules are: - EQ relation conflicts with anything else (including a 2nd EQ)
+ * - Can't have more than one LT(E) relation (resp. GT(E) relation) - IN relation are restricted to row keys
+ * (for now) and conflicts with anything else (we could allow two IN for the same entity but that doesn't seem
+ * very useful) - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value
+ * in CQL so far)
+ */
+ for (Relation relation : whereClause)
+ addRestriction(relation.toRestriction(cfm, boundNames));
+
+ ColumnFamilyStore cfs = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+ SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
+
+ boolean hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(secondaryIndexManager);
+ boolean hasQueriableIndex = hasQueriableClusteringColumnIndex
+ || partitionKeyRestrictions.hasSupportingIndex(secondaryIndexManager)
+ || nonPrimaryKeyRestrictions.hasSupportingIndex(secondaryIndexManager);
+
+ // At this point, the select statement if fully constructed, but we still have a few things to validate
+ processPartitionKeyRestrictions(hasQueriableIndex);
+
+ // Some but not all of the partition key columns have been specified;
+ // hence we need turn these restrictions into index expressions.
+ if (usesSecondaryIndexing)
+ indexRestrictions.add(partitionKeyRestrictions);
+
+ checkFalse(selectsOnlyStaticColumns && hasClusteringColumnsRestriction(),
+ "Cannot restrict clustering columns when selecting only static columns");
+
+ processClusteringColumnsRestrictions(hasQueriableIndex, selectACollection);
+
+ // Covers indexes on the first clustering column (among others).
+ if (isKeyRange && hasQueriableClusteringColumnIndex)
+ usesSecondaryIndexing = true;
+
+ usesSecondaryIndexing = usesSecondaryIndexing || clusteringColumnsRestrictions.isContains();
+
+ if (usesSecondaryIndexing)
+ indexRestrictions.add(clusteringColumnsRestrictions);
+
+ // 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 (!nonPrimaryKeyRestrictions.isEmpty())
+ {
+ usesSecondaryIndexing = true;
+ indexRestrictions.add(nonPrimaryKeyRestrictions);
+ }
+
+ if (usesSecondaryIndexing)
+ validateSecondaryIndexSelections(selectsOnlyStaticColumns);
+ }
+
+ private void addRestriction(Restriction restriction) throws InvalidRequestException
+ {
+ if (restriction.isMultiColumn())
+ clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+ else if (restriction.isOnToken())
+ partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+ else
+ addSingleColumnRestriction((SingleColumnRestriction) restriction);
+ }
+
+ public Iterable<Function> getFunctions()
+ {
+ return Iterables.concat(partitionKeyRestrictions.getFunctions(),
+ clusteringColumnsRestrictions.getFunctions(),
+ nonPrimaryKeyRestrictions.getFunctions());
+ }
+
+ private void addSingleColumnRestriction(SingleColumnRestriction restriction) throws InvalidRequestException
+ {
+ ColumnDefinition def = restriction.columnDef;
+ if (def.isPartitionKey())
+ partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+ else if (def.isClusteringColumn())
+ clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+ else
+ nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction(restriction);
+ }
+
+ /**
+ * Checks if the restrictions on the partition key is an IN restriction.
+ *
+ * @return <code>true</code> the restrictions on the partition key is an IN restriction, <code>false</code>
+ * otherwise.
+ */
+ public boolean keyIsInRelation()
+ {
+ return partitionKeyRestrictions.isIN();
+ }
+
+ /**
+ * Checks if the query request a range of partition keys.
+ *
+ * @return <code>true</code> if the query request a range of partition keys, <code>false</code> otherwise.
+ */
+ public boolean isKeyRange()
+ {
+ return this.isKeyRange;
+ }
+
+ /**
+ * Checks if the secondary index need to be queried.
+ *
+ * @return <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise.
+ */
+ public boolean usesSecondaryIndexing()
+ {
+ return this.usesSecondaryIndexing;
+ }
+
+ private void processPartitionKeyRestrictions(boolean hasQueriableIndex) 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
+ // - Is it queriable without 2ndary index, which is always more efficient
+ // If a component of the partition key is restricted by a relation, all preceding
+ // components must have a EQ. Only the last partition key component can be in IN relation.
+ if (partitionKeyRestrictions.isOnToken())
+ isKeyRange = true;
+
+ if (hasPartitionKeyUnrestrictedComponents())
+ {
+ if (!partitionKeyRestrictions.isEmpty())
+ {
+ if (!hasQueriableIndex)
+ throw invalidRequest("Partition key parts: %s must be restricted as other parts are",
+ Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents()));
+ }
+
+ isKeyRange = true;
+ usesSecondaryIndexing = hasQueriableIndex;
+ }
+ }
+
+ /**
+ * Checks if the partition key has some unrestricted components.
+ * @return <code>true</code> if the partition key has some unrestricted components, <code>false</code> otherwise.
+ */
+ private boolean hasPartitionKeyUnrestrictedComponents()
+ {
+ return partitionKeyRestrictions.size() < cfm.partitionKeyColumns().size();
+ }
+
+ /**
+ * Returns the partition key components that are not restricted.
+ * @return the partition key components that are not restricted.
+ */
+ private List<ColumnIdentifier> getPartitionKeyUnrestrictedComponents()
+ {
+ List<ColumnDefinition> list = new ArrayList<>(cfm.partitionKeyColumns());
+ list.removeAll(partitionKeyRestrictions.getColumnDefs());
+ return ColumnDefinition.toIdentifiers(list);
+ }
+
+ /**
+ * Processes the clustering column restrictions.
+ *
+ * @param hasQueriableIndex <code>true</code> if some of the queried data are indexed, <code>false</code> otherwise
+ * @param selectACollection <code>true</code> if the query should return a collection column
+ * @throws InvalidRequestException if the request is invalid
+ */
+ private void processClusteringColumnsRestrictions(boolean hasQueriableIndex,
+ boolean selectACollection) throws InvalidRequestException
+ {
+ checkFalse(clusteringColumnsRestrictions.isIN() && selectACollection,
+ "Cannot restrict clustering columns by IN relations when a collection is selected by the query");
+ checkFalse(clusteringColumnsRestrictions.isContains() && !hasQueriableIndex,
+ "Cannot restrict clustering columns by a CONTAINS relation without a secondary index");
+
+ if (hasClusteringColumnsRestriction())
+ {
+ List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
+ List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
+
+ for (int i = 0, m = restrictedColumns.size(); i < m; i++)
+ {
+ ColumnDefinition clusteringColumn = clusteringColumns.get(i);
+ ColumnDefinition restrictedColumn = restrictedColumns.get(i);
+
+ if (!clusteringColumn.equals(restrictedColumn))
+ {
+ checkTrue(hasQueriableIndex,
+ "PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted",
+ restrictedColumn.name,
+ clusteringColumn.name);
+
+ usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
+ break;
+ }
+ }
+ }
+
+ if (clusteringColumnsRestrictions.isContains())
+ usesSecondaryIndexing = true;
+ }
+
+ public List<IndexExpression> getIndexExpressions(SecondaryIndexManager indexManager,
+ QueryOptions options) throws InvalidRequestException
+ {
+ if (!usesSecondaryIndexing || indexRestrictions.isEmpty())
+ return Collections.emptyList();
+
+ List<IndexExpression> expressions = new ArrayList<>();
+ for (Restrictions restrictions : indexRestrictions)
+ restrictions.addIndexExpressionTo(expressions, indexManager, options);
+
+ return expressions;
+ }
+
+ /**
+ * Returns the partition keys for which the data is requested.
+ *
+ * @param options the query options
+ * @return the partition keys for which the data is requested.
+ * @throws InvalidRequestException if the partition keys cannot be retrieved
+ */
+ public Collection<ByteBuffer> getPartitionKeys(final QueryOptions options) throws InvalidRequestException
+ {
+ return partitionKeyRestrictions.values(options);
+ }
+
+ /**
+ * Returns the specified bound of the partition key.
+ *
+ * @param b the boundary type
+ * @param options the query options
+ * @return the specified bound of the partition key
+ * @throws InvalidRequestException if the boundary cannot be retrieved
+ */
+ private ByteBuffer getPartitionKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
+ {
+ // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the
+ // first
+ // component of a composite partition key).
+ if (hasPartitionKeyUnrestrictedComponents())
+ return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+ // We deal with IN queries for keys in other places, so we know buildBound will return only one result
+ return partitionKeyRestrictions.bounds(b, options).get(0);
+ }
+
+ /**
+ * Returns the partition key bounds.
+ *
+ * @param options the query options
+ * @return the partition key bounds
+ * @throws InvalidRequestException if the query is invalid
+ */
+ public AbstractBounds<RowPosition> getPartitionKeyBounds(QueryOptions options) throws InvalidRequestException
+ {
+ IPartitioner p = StorageService.getPartitioner();
+
+ if (partitionKeyRestrictions.isOnToken())
+ {
+ return getPartitionKeyBoundsForTokenRestrictions(p, options);
+ }
+
+ return getPartitionKeyBounds(p, options);
+ }
+
+ private AbstractBounds<RowPosition> getPartitionKeyBounds(IPartitioner p,
+ QueryOptions options) throws InvalidRequestException
+ {
+ ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options);
+ ByteBuffer finishKeyBytes = getPartitionKeyBound(Bound.END, options);
+
+ RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
+ RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
+
+ if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum())
+ return null;
+
+ if (partitionKeyRestrictions.isInclusive(Bound.START))
+ {
+ return partitionKeyRestrictions.isInclusive(Bound.END)
+ ? new Bounds<>(startKey, finishKey)
+ : new IncludingExcludingBounds<>(startKey, finishKey);
+ }
+
+ return partitionKeyRestrictions.isInclusive(Bound.END)
+ ? new Range<>(startKey, finishKey)
+ : new ExcludingBounds<>(startKey, finishKey);
+ }
+
+ private AbstractBounds<RowPosition> getPartitionKeyBoundsForTokenRestrictions(IPartitioner p,
+ QueryOptions options)
+ throws InvalidRequestException
+ {
+ Token startToken = getTokenBound(Bound.START, options, p);
+ Token endToken = getTokenBound(Bound.END, options, p);
+
+ boolean includeStart = partitionKeyRestrictions.isInclusive(Bound.START);
+ boolean includeEnd = partitionKeyRestrictions.isInclusive(Bound.END);
+
+ /*
+ * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
+ * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result in that
+ * case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
+ *
+ * In practice, we want to return an empty result set if either startToken > endToken, or both are equal but
+ * one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a) or (a, a)).
+ * Note though that in the case where startToken or endToken is the minimum token, then this special case
+ * rule should not apply.
+ */
+ int cmp = startToken.compareTo(endToken);
+ if (!startToken.isMinimum() && !endToken.isMinimum()
+ && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
+ return null;
+
+ RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
+ RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
+
+ return new Range<>(start, end);
+ }
+
+ private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
+ {
+ if (!partitionKeyRestrictions.hasBound(b))
+ return p.getMinimumToken();
+
+ ByteBuffer value = partitionKeyRestrictions.bounds(b, options).get(0);
+ checkNotNull(value, "Invalid null token value");
+ return p.getTokenFactory().fromByteArray(value);
+ }
+
+ /**
+ * Checks if the query does not contains any restriction on the clustering columns.
+ *
+ * @return <code>true</code> if the query does not contains any restriction on the clustering columns,
+ * <code>false</code> otherwise.
+ */
+ public boolean hasNoClusteringColumnsRestriction()
+ {
+ return clusteringColumnsRestrictions.isEmpty();
+ }
+
++ /**
++ * Checks if the query has some restrictions on the clustering columns.
++ *
++ * @return <code>true</code> if the query has some restrictions on the clustering columns,
++ * <code>false</code> otherwise.
++ */
++ public boolean hasClusteringColumnsRestriction()
++ {
++ return !clusteringColumnsRestrictions.isEmpty();
++ }
++
+ // For non-composite slices, we don't support internally the difference between exclusive and
+ // inclusive bounds, so we deal with it manually.
+ public boolean isNonCompositeSliceWithExclusiveBounds()
+ {
+ return !cfm.comparator.isCompound()
+ && clusteringColumnsRestrictions.isSlice()
+ && (!clusteringColumnsRestrictions.isInclusive(Bound.START) || !clusteringColumnsRestrictions.isInclusive(Bound.END));
+ }
+
+ /**
+ * Returns the requested clustering columns as <code>Composite</code>s.
+ *
+ * @param options the query options
+ * @return the requested clustering columns as <code>Composite</code>s
+ * @throws InvalidRequestException if the query is not valid
+ */
+ public List<Composite> getClusteringColumnsAsComposites(QueryOptions options) throws InvalidRequestException
+ {
+ return clusteringColumnsRestrictions.valuesAsComposites(options);
+ }
+
+ /**
+ * Returns the bounds (start or end) of the clustering columns as <code>Composites</code>.
+ *
+ * @param b the bound type
+ * @param options the query options
+ * @return the bounds (start or end) of the clustering columns as <code>Composites</code>
+ * @throws InvalidRequestException if the request is not valid
+ */
+ public List<Composite> getClusteringColumnsBoundsAsComposites(Bound b,
+ QueryOptions options) throws InvalidRequestException
+ {
+ return clusteringColumnsRestrictions.boundsAsComposites(b, options);
+ }
+
+ /**
+ * Returns the bounds (start or end) of the clustering columns.
+ *
+ * @param b the bound type
+ * @param options the query options
+ * @return the bounds (start or end) of the clustering columns
+ * @throws InvalidRequestException if the request is not valid
+ */
+ public List<ByteBuffer> getClusteringColumnsBounds(Bound b, QueryOptions options) throws InvalidRequestException
+ {
+ return clusteringColumnsRestrictions.bounds(b, options);
+ }
+
+ /**
+ * Checks if the bounds (start or end) of the clustering columns are inclusive.
+ *
+ * @param bound the bound type
+ * @return <code>true</code> if the bounds (start or end) of the clustering columns are inclusive,
+ * <code>false</code> otherwise
+ */
+ public boolean areRequestedBoundsInclusive(Bound bound)
+ {
+ return clusteringColumnsRestrictions.isInclusive(bound);
+ }
+
+ /**
+ * Checks if the query returns a range of columns.
+ *
+ * @return <code>true</code> if the query returns a range of columns, <code>false</code> otherwise.
+ */
+ public boolean isColumnRange()
+ {
+ // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
+ // Static CF (non dense but non composite) never entails a column slice however
+ if (!cfm.comparator.isDense())
+ return cfm.comparator.isCompound();
+
+ // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about
+ // CASSANDRA-5762),
+ // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
+ return clusteringColumnsRestrictions.size() < cfm.clusteringColumns().size() || clusteringColumnsRestrictions.isSlice();
+ }
+
+ /**
+ * Checks if the query need to use filtering.
+ * @return <code>true</code> if the query need to use filtering, <code>false</code> otherwise.
+ */
+ public boolean needFiltering()
+ {
+ int numberOfRestrictedColumns = 0;
+ for (Restrictions restrictions : indexRestrictions)
+ numberOfRestrictedColumns += restrictions.size();
+
+ return numberOfRestrictedColumns > 1
+ || (numberOfRestrictedColumns == 0 && !clusteringColumnsRestrictions.isEmpty())
+ || (numberOfRestrictedColumns != 0
+ && nonPrimaryKeyRestrictions.hasMultipleContains());
+ }
+
+ private void validateSecondaryIndexSelections(boolean selectsOnlyStaticColumns) throws InvalidRequestException
+ {
+ checkFalse(keyIsInRelation(),
+ "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.
+ checkFalse(selectsOnlyStaticColumns, "Queries using 2ndary indexes don't support selecting only static columns");
+ }
+
- /**
- * Checks if the query has some restrictions on the clustering columns.
- *
- * @return <code>true</code> if the query has some restrictions on the clustering columns,
- * <code>false</code> otherwise.
- */
- private boolean hasClusteringColumnsRestriction()
- {
- return !clusteringColumnsRestrictions.isEmpty();
- }
-
+ public void reverse()
+ {
+ clusteringColumnsRestrictions = new ReversedPrimaryKeyRestrictions(clusteringColumnsRestrictions);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index 1dd1903,0000000..fa40152
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@@ -1,120 -1,0 +1,133 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import com.google.common.collect.Iterables;
+import org.apache.commons.lang3.text.StrBuilder;
+
++import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+abstract class AbstractFunctionSelector<T extends Function> extends Selector
+{
+ protected final T fun;
+
+ /**
+ * The list used to pass the function arguments is recycled to avoid the cost of instantiating a new list
+ * with each function call.
+ */
+ protected final List<ByteBuffer> args;
+ protected final List<Selector> argSelectors;
+
+ public static Factory newFactory(final Function fun, final SelectorFactories factories) throws InvalidRequestException
+ {
+ if (fun.isAggregate())
+ {
+ if (factories.doesAggregation())
+ throw new InvalidRequestException("aggregate functions cannot be used as arguments of aggregate functions");
+ }
+ else
+ {
+ if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
+ throw new InvalidRequestException(String.format("arguments of function %s must be either all aggregates or no aggregates",
+ fun.name()));
+ }
+
+ return new Factory()
+ {
+ protected String getColumnName()
+ {
+ return new StrBuilder(fun.name().toString()).append('(')
+ .appendWithSeparators(factories.getColumnNames(), ", ")
+ .append(')')
+ .toString();
+ }
+
+ protected AbstractType<?> getReturnType()
+ {
+ return fun.returnType();
+ }
+
++ protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
++ {
++ for (Factory factory : factories)
++ factory.addColumnMapping(mapping, resultsColumn);
++
++ if (mapping.getMappings().get(resultsColumn).isEmpty())
++ // add a null mapping for cases where there are no
++ // further selectors, such as no-arg functions and count
++ mapping.addMapping(resultsColumn, null);
++
++ }
++
+ public Iterable<Function> getFunctions()
+ {
+ return Iterables.concat(fun.getFunctions(), factories.getFunctions());
+ }
+
+ public Selector newInstance() throws InvalidRequestException
+ {
+ return fun.isAggregate() ? new AggregateFunctionSelector(fun, factories.newInstances())
+ : new ScalarFunctionSelector(fun, factories.newInstances());
+ }
+
+ public boolean isWritetimeSelectorFactory()
+ {
+ return factories.containsWritetimeSelectorFactory();
+ }
+
+ public boolean isTTLSelectorFactory()
+ {
+ return factories.containsTTLSelectorFactory();
+ }
+
+ public boolean isAggregateSelectorFactory()
+ {
+ return fun.isAggregate() || factories.containsOnlyAggregateFunctions();
+ }
+ };
+ }
+
+ protected AbstractFunctionSelector(T fun, List<Selector> argSelectors)
+ {
+ this.fun = fun;
+ this.argSelectors = argSelectors;
+ this.args = Arrays.asList(new ByteBuffer[argSelectors.size()]);
+ }
+
+ public AbstractType<?> getType()
+ {
+ return fun.returnType();
+ }
+
+ @Override
+ public String toString()
+ {
+ return new StrBuilder().append(fun.name())
+ .append("(")
+ .appendWithSeparators(argSelectors, ", ")
+ .append(")")
+ .toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
index 76dbb22,0000000..63b6cc6
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
@@@ -1,103 -1,0 +1,109 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+
++import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+final class FieldSelector extends Selector
+{
+ private final UserType type;
+ private final int field;
+ private final Selector selected;
+
+ public static Factory newFactory(final UserType type, final int field, final Selector.Factory factory)
+ {
+ return new Factory()
+ {
+ protected String getColumnName()
+ {
+ return String.format("%s.%s",
+ factory.getColumnName(),
+ UTF8Type.instance.getString(type.fieldName(field)));
+ }
+
+ protected AbstractType<?> getReturnType()
+ {
+ return type.fieldType(field);
+ }
+
++ protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
++ {
++ factory.addColumnMapping(mapping, resultsColumn);
++ }
++
+ public Selector newInstance() throws InvalidRequestException
+ {
+ return new FieldSelector(type, field, factory.newInstance());
+ }
+
+ public boolean isAggregateSelectorFactory()
+ {
+ return factory.isAggregateSelectorFactory();
+ }
+ };
+ }
+
+ public boolean isAggregate()
+ {
+ return false;
+ }
+
+ public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+ {
+ selected.addInput(protocolVersion, rs);
+ }
+
+ public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+ {
+ ByteBuffer value = selected.getOutput(protocolVersion);
+ if (value == null)
+ return null;
+ ByteBuffer[] buffers = type.split(value);
+ return field < buffers.length ? buffers[field] : null;
+ }
+
+ public AbstractType<?> getType()
+ {
+ return type.fieldType(field);
+ }
+
+ public void reset()
+ {
+ selected.reset();
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("%s.%s", selected, UTF8Type.instance.getString(type.fieldName(field)));
+ }
+
+ private FieldSelector(UserType type, int field, Selector selected)
+ {
+ this.type = type;
+ this.field = field;
+ this.selected = selected;
+ }
- }
++}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 4506111,0000000..ee134ee
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@@ -1,253 -1,0 +1,251 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.util.ArrayList;
+import java.util.List;
+
++import org.apache.commons.lang3.text.StrBuilder;
++
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
- import org.apache.cassandra.cql3.functions.Function;
- import org.apache.cassandra.cql3.functions.FunctionName;
- import org.apache.cassandra.cql3.functions.Functions;
- import org.apache.cassandra.cql3.functions.ToJsonFct;
++import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
- import org.apache.commons.lang3.text.StrBuilder;
+
+public abstract class Selectable
+{
+ public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
+ throws InvalidRequestException;
+
+ protected static int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
+ {
+ int idx = l.indexOf(def);
+ if (idx < 0)
+ {
+ idx = l.size();
+ l.add(def);
+ }
+ return idx;
+ }
+
+ public static interface Raw
+ {
+ public Selectable prepare(CFMetaData cfm);
+
+ /**
+ * Returns true if any processing is performed on the selected column.
+ **/
+ public boolean processesSelection();
+ }
+
+ public static class WritetimeOrTTL extends Selectable
+ {
+ public final ColumnIdentifier id;
+ public final boolean isWritetime;
+
+ public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
+ {
+ this.id = id;
+ this.isWritetime = isWritetime;
+ }
+
+ @Override
+ public String toString()
+ {
+ return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
+ }
+
+ public Selector.Factory newSelectorFactory(CFMetaData cfm,
+ List<ColumnDefinition> defs) throws InvalidRequestException
+ {
+ ColumnDefinition def = cfm.getColumnDefinition(id);
+ if (def == null)
+ throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
+ if (def.isPrimaryKeyColumn())
+ throw new InvalidRequestException(
+ String.format("Cannot use selection function %s on PRIMARY KEY part %s",
+ isWritetime ? "writeTime" : "ttl",
+ def.name));
+ if (def.type.isCollection())
+ throw new InvalidRequestException(String.format("Cannot use selection function %s on collections",
+ isWritetime ? "writeTime" : "ttl"));
+
- return WritetimeOrTTLSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs), isWritetime);
++ return WritetimeOrTTLSelector.newFactory(def, addAndGetIndex(def, defs), isWritetime);
+ }
+
+ public static class Raw implements Selectable.Raw
+ {
+ private final ColumnIdentifier.Raw id;
+ private final boolean isWritetime;
+
+ public Raw(ColumnIdentifier.Raw id, boolean isWritetime)
+ {
+ this.id = id;
+ this.isWritetime = isWritetime;
+ }
+
+ public WritetimeOrTTL prepare(CFMetaData cfm)
+ {
+ return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
+ }
+
+ public boolean processesSelection()
+ {
+ return true;
+ }
+ }
+ }
+
+ public static class WithFunction extends Selectable
+ {
+ public final FunctionName functionName;
+ public final List<Selectable> args;
+
+ public WithFunction(FunctionName functionName, List<Selectable> args)
+ {
+ this.functionName = functionName;
+ this.args = args;
+ }
+
+ @Override
+ public String toString()
+ {
+ return new StrBuilder().append(functionName)
+ .append("(")
+ .appendWithSeparators(args, ", ")
+ .append(")")
+ .toString();
+ }
+
+ public Selector.Factory newSelectorFactory(CFMetaData cfm,
+ List<ColumnDefinition> defs) throws InvalidRequestException
+ {
+ SelectorFactories factories =
+ SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, cfm, defs);
+
+ // We need to circumvent the normal function lookup process for toJson() because instances of the function
+ // are not pre-declared (because it can accept any type of argument).
+ Function fun;
+ if (functionName.equalsNativeFunction(ToJsonFct.NAME))
+ fun = ToJsonFct.getInstance(factories.getReturnTypes());
+ else
+ fun = Functions.get(cfm.ksName, functionName, factories.newInstances(), cfm.ksName, cfm.cfName, null);
+
+ if (fun == null)
+ throw new InvalidRequestException(String.format("Unknown function '%s'", functionName));
+ if (fun.returnType() == null)
+ throw new InvalidRequestException(String.format("Unknown function %s called in selection clause",
+ functionName));
+
+ return AbstractFunctionSelector.newFactory(fun, factories);
+ }
+
+ public static class Raw implements Selectable.Raw
+ {
+ private final FunctionName functionName;
+ private final List<Selectable.Raw> args;
+
+ public Raw(FunctionName functionName, List<Selectable.Raw> args)
+ {
+ this.functionName = functionName;
+ this.args = args;
+ }
+
+ public WithFunction prepare(CFMetaData cfm)
+ {
+ List<Selectable> preparedArgs = new ArrayList<>(args.size());
+ for (Selectable.Raw arg : args)
+ preparedArgs.add(arg.prepare(cfm));
+ return new WithFunction(functionName, preparedArgs);
+ }
+
+ public boolean processesSelection()
+ {
+ return true;
+ }
+ }
+ }
+
+ public static class WithFieldSelection extends Selectable
+ {
+ public final Selectable selected;
+ public final ColumnIdentifier field;
+
+ public WithFieldSelection(Selectable selected, ColumnIdentifier field)
+ {
+ this.selected = selected;
+ this.field = field;
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("%s.%s", selected, field);
+ }
+
+ public Selector.Factory newSelectorFactory(CFMetaData cfm,
+ List<ColumnDefinition> defs) throws InvalidRequestException
+ {
+ Selector.Factory factory = selected.newSelectorFactory(cfm, defs);
+ AbstractType<?> type = factory.newInstance().getType();
+ if (!(type instanceof UserType))
+ throw new InvalidRequestException(
+ String.format("Invalid field selection: %s of type %s is not a user type",
+ selected,
+ type.asCQL3Type()));
+
+ UserType ut = (UserType) type;
+ for (int i = 0; i < ut.size(); i++)
+ {
+ if (!ut.fieldName(i).equals(field.bytes))
+ continue;
+ return FieldSelector.newFactory(ut, i, factory);
+ }
+ throw new InvalidRequestException(String.format("%s of type %s has no field %s",
+ selected,
+ type.asCQL3Type(),
+ field));
+ }
+
+ public static class Raw implements Selectable.Raw
+ {
+ private final Selectable.Raw selected;
+ private final ColumnIdentifier.Raw field;
+
+ public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field)
+ {
+ this.selected = selected;
+ this.field = field;
+ }
+
+ public WithFieldSelection prepare(CFMetaData cfm)
+ {
+ return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm));
+ }
+
+ public boolean processesSelection()
+ {
+ return true;
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selection.java
index 9c990ce,0000000..25278df
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@@ -1,535 -1,0 +1,547 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.CounterCell;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public abstract class Selection
+{
+ /**
+ * A predicate that returns <code>true</code> for static columns.
+ */
+ private static final Predicate<ColumnDefinition> STATIC_COLUMN_FILTER = new Predicate<ColumnDefinition>()
+ {
+ public boolean apply(ColumnDefinition def)
+ {
+ return def.isStatic();
+ }
+ };
+
+ private final CFMetaData cfm;
- private final Collection<ColumnDefinition> columns;
++ private final List<ColumnDefinition> columns;
++ private final SelectionColumnMapping columnMapping;
+ private final ResultSet.ResultMetadata metadata;
+ private final boolean collectTimestamps;
+ private final boolean collectTTLs;
+
+ protected Selection(CFMetaData cfm,
- Collection<ColumnDefinition> columns,
- List<ColumnSpecification> metadata,
++ List<ColumnDefinition> columns,
++ SelectionColumnMapping columnMapping,
+ boolean collectTimestamps,
+ boolean collectTTLs)
+ {
+ this.cfm = cfm;
+ this.columns = columns;
- this.metadata = new ResultSet.ResultMetadata(metadata);
++ this.columnMapping = columnMapping;
++ this.metadata = new ResultSet.ResultMetadata(columnMapping.getColumnSpecifications());
+ this.collectTimestamps = collectTimestamps;
+ this.collectTTLs = collectTTLs;
+ }
+
+ // Overriden by SimpleSelection when appropriate.
+ public boolean isWildcard()
+ {
+ return false;
+ }
+
+ /**
+ * Checks if this selection contains static columns.
+ * @return <code>true</code> if this selection contains static columns, <code>false</code> otherwise;
+ */
+ public boolean containsStaticColumns()
+ {
+ if (!cfm.hasStaticColumns())
+ return false;
+
+ if (isWildcard())
+ return true;
+
+ return !Iterables.isEmpty(Iterables.filter(columns, STATIC_COLUMN_FILTER));
+ }
+
+ /**
+ * Checks if this selection contains only static columns.
+ * @return <code>true</code> if this selection contains only static columns, <code>false</code> otherwise;
+ */
+ public boolean containsOnlyStaticColumns()
+ {
+ if (!containsStaticColumns())
+ return false;
+
+ if (isWildcard())
+ return false;
+
+ for (ColumnDefinition def : getColumns())
+ {
+ if (!def.isPartitionKey() && !def.isStatic())
+ return false;
+ }
+
+ return true;
+ }
+
+ /**
+ * Checks if this selection contains a collection.
+ *
+ * @return <code>true</code> if this selection contains a collection, <code>false</code> otherwise.
+ */
+ public boolean containsACollection()
+ {
+ if (!cfm.comparator.hasCollections())
+ return false;
+
+ for (ColumnDefinition def : getColumns())
+ if (def.type.isCollection() && def.type.isMultiCell())
+ return true;
+
+ return false;
+ }
+
+ /**
+ * Returns the index of the specified column.
+ *
+ * @param def the column definition
+ * @return the index of the specified column
+ */
+ public int indexOf(final ColumnDefinition def)
+ {
+ return Iterators.indexOf(getColumns().iterator(), new Predicate<ColumnDefinition>()
+ {
+ public boolean apply(ColumnDefinition n)
+ {
+ return def.name.equals(n.name);
+ }
+ });
+ }
+
+ public ResultSet.ResultMetadata getResultMetadata(boolean isJson)
+ {
+ if (!isJson)
+ return metadata;
+
+ ColumnSpecification firstColumn = metadata.names.get(0);
+ ColumnSpecification jsonSpec = new ColumnSpecification(firstColumn.ksName, firstColumn.cfName, Json.JSON_COLUMN_ID, UTF8Type.instance);
+ return new ResultSet.ResultMetadata(Arrays.asList(jsonSpec));
+ }
+
+ public static Selection wildcard(CFMetaData cfm)
+ {
- List<ColumnDefinition> all = new ArrayList<ColumnDefinition>(cfm.allColumns().size());
++ List<ColumnDefinition> all = new ArrayList<>(cfm.allColumns().size());
+ Iterators.addAll(all, cfm.allColumnsInSelectOrder());
+ return new SimpleSelection(cfm, all, true);
+ }
+
- public static Selection forColumns(CFMetaData cfm, Collection<ColumnDefinition> columns)
++ public static Selection forColumns(CFMetaData cfm, List<ColumnDefinition> columns)
+ {
+ return new SimpleSelection(cfm, columns, false);
+ }
+
+ public int addColumnForOrdering(ColumnDefinition c)
+ {
+ columns.add(c);
+ metadata.addNonSerializedColumn(c);
+ return columns.size() - 1;
+ }
+
+ public Iterable<Function> getFunctions()
+ {
+ return Collections.emptySet();
+ }
+
+ private static boolean processesSelection(List<RawSelector> rawSelectors)
+ {
+ for (RawSelector rawSelector : rawSelectors)
+ {
+ if (rawSelector.processesSelection())
+ return true;
+ }
+ return false;
+ }
+
+ public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors) throws InvalidRequestException
+ {
- List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>();
++ List<ColumnDefinition> defs = new ArrayList<>();
+
+ SelectorFactories factories =
+ SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), cfm, defs);
- List<ColumnSpecification> metadata = collectMetadata(cfm, rawSelectors, factories);
++ SelectionColumnMapping mapping = collectColumnMappings(cfm, rawSelectors, factories);
+
- return processesSelection(rawSelectors) ? new SelectionWithProcessing(cfm, defs, metadata, factories)
- : new SimpleSelection(cfm, defs, metadata, false);
++ return processesSelection(rawSelectors) ? new SelectionWithProcessing(cfm, defs, mapping, factories)
++ : new SimpleSelection(cfm, defs, mapping, false);
+ }
+
- private static List<ColumnSpecification> collectMetadata(CFMetaData cfm,
- List<RawSelector> rawSelectors,
- SelectorFactories factories)
++ private static SelectionColumnMapping collectColumnMappings(CFMetaData cfm,
++ List<RawSelector> rawSelectors,
++ SelectorFactories factories)
+ {
- List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
++ SelectionColumnMapping selectionColumns = SelectionColumnMapping.newMapping();
+ Iterator<RawSelector> iter = rawSelectors.iterator();
+ for (Selector.Factory factory : factories)
+ {
+ ColumnSpecification colSpec = factory.getColumnSpecification(cfm);
+ ColumnIdentifier alias = iter.next().alias;
- metadata.add(alias == null ? colSpec : colSpec.withAlias(alias));
++ factory.addColumnMapping(selectionColumns,
++ alias == null ? colSpec : colSpec.withAlias(alias));
+ }
- return metadata;
++ return selectionColumns;
+ }
+
+ protected abstract Selectors newSelectors() throws InvalidRequestException;
+
+ /**
+ * @return the list of CQL3 columns value this SelectionClause needs.
+ */
- public Collection<ColumnDefinition> getColumns()
++ public List<ColumnDefinition> getColumns()
+ {
+ return columns;
+ }
+
++ /**
++ * @return the mappings between resultset columns and the underlying columns
++ */
++ public SelectionColumns getColumnMapping()
++ {
++ return columnMapping;
++ }
++
+ public ResultSetBuilder resultSetBuilder(long now, boolean isJson) throws InvalidRequestException
+ {
+ return new ResultSetBuilder(now, isJson);
+ }
+
+ public abstract boolean isAggregate();
+
+ @Override
+ public String toString()
+ {
+ return Objects.toStringHelper(this)
+ .add("columns", columns)
++ .add("columnMapping", columnMapping)
+ .add("metadata", metadata)
+ .add("collectTimestamps", collectTimestamps)
+ .add("collectTTLs", collectTTLs)
+ .toString();
+ }
+
+ public class ResultSetBuilder
+ {
+ private final ResultSet resultSet;
+
+ /**
+ * As multiple thread can access a <code>Selection</code> instance each <code>ResultSetBuilder</code> will use
+ * its own <code>Selectors</code> instance.
+ */
+ private final Selectors selectors;
+
+ /*
+ * We'll build CQL3 row one by one.
+ * The currentRow is the values for the (CQL3) columns we've fetched.
+ * We also collect timestamps and ttls for the case where the writetime and
+ * ttl functions are used. Note that we might collect timestamp and/or ttls
+ * we don't care about, but since the array below are allocated just once,
+ * it doesn't matter performance wise.
+ */
+ List<ByteBuffer> current;
+ final long[] timestamps;
+ final int[] ttls;
+ final long now;
+
+ private final boolean isJson;
+
+ private ResultSetBuilder(long now, boolean isJson) throws InvalidRequestException
+ {
+ this.resultSet = new ResultSet(getResultMetadata(isJson).copy(), new ArrayList<List<ByteBuffer>>());
+ this.selectors = newSelectors();
+ this.timestamps = collectTimestamps ? new long[columns.size()] : null;
+ this.ttls = collectTTLs ? new int[columns.size()] : null;
+ this.now = now;
+ this.isJson = isJson;
+ }
+
+ public void add(ByteBuffer v)
+ {
+ current.add(v);
+ }
+
+ public void add(Cell c)
+ {
+ current.add(isDead(c) ? null : value(c));
+ if (timestamps != null)
+ {
+ timestamps[current.size() - 1] = isDead(c) ? Long.MIN_VALUE : c.timestamp();
+ }
+ if (ttls != null)
+ {
+ int ttl = -1;
+ if (!isDead(c) && c instanceof ExpiringCell)
+ ttl = c.getLocalDeletionTime() - (int) (now / 1000);
+ ttls[current.size() - 1] = ttl;
+ }
+ }
+
+ private boolean isDead(Cell c)
+ {
+ return c == null || !c.isLive(now);
+ }
+
+ public void newRow(int protocolVersion) throws InvalidRequestException
+ {
+ if (current != null)
+ {
+ selectors.addInputRow(protocolVersion, this);
+ if (!selectors.isAggregate())
+ {
+ resultSet.addRow(getOutputRow(protocolVersion));
+ selectors.reset();
+ }
+ }
+ current = new ArrayList<>(columns.size());
+ }
+
+ public ResultSet build(int protocolVersion) throws InvalidRequestException
+ {
+ if (current != null)
+ {
+ selectors.addInputRow(protocolVersion, this);
+ resultSet.addRow(getOutputRow(protocolVersion));
+ selectors.reset();
+ current = null;
+ }
+
+ if (resultSet.isEmpty() && selectors.isAggregate())
+ resultSet.addRow(getOutputRow(protocolVersion));
+ return resultSet;
+ }
+
+ private List<ByteBuffer> getOutputRow(int protocolVersion)
+ {
+ List<ByteBuffer> outputRow = selectors.getOutputRow(protocolVersion);
+ return isJson ? rowToJson(outputRow, protocolVersion)
+ : outputRow;
+ }
+
+ private List<ByteBuffer> rowToJson(List<ByteBuffer> row, int protocolVersion)
+ {
+ StringBuilder sb = new StringBuilder("{");
+ for (int i = 0; i < metadata.names.size(); i++)
+ {
+ if (i > 0)
+ sb.append(", ");
+
+ ColumnSpecification spec = metadata.names.get(i);
+ String columnName = spec.name.toString();
+ if (!columnName.equals(columnName.toLowerCase(Locale.US)))
+ columnName = "\"" + columnName + "\"";
+
+ ByteBuffer buffer = row.get(i);
+ sb.append('"');
+ sb.append(Json.JSON_STRING_ENCODER.quoteAsString(columnName));
+ sb.append("\": ");
+ if (buffer == null)
+ sb.append("null");
+ else
+ sb.append(spec.type.toJSONString(buffer, protocolVersion));
+ }
+ sb.append("}");
+ return Collections.singletonList(UTF8Type.instance.getSerializer().serialize(sb.toString()));
+ }
+
+ private ByteBuffer value(Cell c)
+ {
+ return (c instanceof CounterCell)
+ ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
+ : c.value();
+ }
+ }
+
+ private static interface Selectors
+ {
+ public boolean isAggregate();
+
+ /**
+ * Adds the current row of the specified <code>ResultSetBuilder</code>.
+ *
+ * @param rs the <code>ResultSetBuilder</code>
+ * @throws InvalidRequestException
+ */
+ public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException;
+
+ public List<ByteBuffer> getOutputRow(int protocolVersion) throws InvalidRequestException;
+
+ public void reset();
+ }
+
+ // Special cased selection for when no function is used (this save some allocations).
+ private static class SimpleSelection extends Selection
+ {
+ private final boolean isWildcard;
+
- public SimpleSelection(CFMetaData cfm, Collection<ColumnDefinition> columns, boolean isWildcard)
++ public SimpleSelection(CFMetaData cfm, List<ColumnDefinition> columns, boolean isWildcard)
+ {
- this(cfm, columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
++ this(cfm, columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
+ }
+
+ public SimpleSelection(CFMetaData cfm,
- Collection<ColumnDefinition> columns,
- List<ColumnSpecification> metadata,
++ List<ColumnDefinition> columns,
++ SelectionColumnMapping metadata,
+ boolean isWildcard)
+ {
+ /*
+ * In theory, even a simple selection could have multiple time the same column, so we
+ * could filter those duplicate out of columns. But since we're very unlikely to
+ * get much duplicate in practice, it's more efficient not to bother.
+ */
+ super(cfm, columns, metadata, false, false);
+ this.isWildcard = isWildcard;
+ }
+
+ @Override
+ public boolean isWildcard()
+ {
+ return isWildcard;
+ }
+
+ public boolean isAggregate()
+ {
+ return false;
+ }
+
+ protected Selectors newSelectors()
+ {
+ return new Selectors()
+ {
+ private List<ByteBuffer> current;
+
+ public void reset()
+ {
+ current = null;
+ }
+
+ public List<ByteBuffer> getOutputRow(int protocolVersion)
+ {
+ return current;
+ }
+
+ public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+ {
+ current = rs.current;
+ }
+
+ public boolean isAggregate()
+ {
+ return false;
+ }
+ };
+ }
+ }
+
+ private static class SelectionWithProcessing extends Selection
+ {
+ private final SelectorFactories factories;
+
+ public SelectionWithProcessing(CFMetaData cfm,
- Collection<ColumnDefinition> columns,
- List<ColumnSpecification> metadata,
++ List<ColumnDefinition> columns,
++ SelectionColumnMapping metadata,
+ SelectorFactories factories) throws InvalidRequestException
+ {
+ super(cfm,
+ columns,
+ metadata,
+ factories.containsWritetimeSelectorFactory(),
+ factories.containsTTLSelectorFactory());
+
+ this.factories = factories;
+
+ if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
+ throw new InvalidRequestException("the select clause must either contain only aggregates or no aggregate");
+ }
+
+ @Override
+ public Iterable<Function> getFunctions()
+ {
+ return factories.getFunctions();
+ }
+
+ @Override
+ public int addColumnForOrdering(ColumnDefinition c)
+ {
+ int index = super.addColumnForOrdering(c);
+ factories.addSelectorForOrdering(c, index);
+ return index;
+ }
+
+ public boolean isAggregate()
+ {
+ return factories.containsOnlyAggregateFunctions();
+ }
+
+ protected Selectors newSelectors() throws InvalidRequestException
+ {
+ return new Selectors()
+ {
+ private final List<Selector> selectors = factories.newInstances();
+
+ public void reset()
+ {
+ for (Selector selector : selectors)
+ selector.reset();
+ }
+
+ public boolean isAggregate()
+ {
+ return factories.containsOnlyAggregateFunctions();
+ }
+
+ public List<ByteBuffer> getOutputRow(int protocolVersion) throws InvalidRequestException
+ {
+ List<ByteBuffer> outputRow = new ArrayList<>(selectors.size());
+
+ for (Selector selector: selectors)
+ outputRow.add(selector.getOutput(protocolVersion));
+
+ return outputRow;
+ }
+
+ public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+ {
+ for (Selector selector : selectors)
+ selector.addInput(protocolVersion, rs);
+ }
+ };
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
index 0000000,0000000..e6c8979
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
@@@ -1,0 -1,0 +1,118 @@@
++package org.apache.cassandra.cql3.selection;
++
++import java.util.LinkedHashSet;
++import java.util.List;
++
++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.config.ColumnDefinition;
++import org.apache.cassandra.cql3.ColumnSpecification;
++
++/**
++ * Separately maintains the ColumnSpecifications and their mappings to underlying
++ * columns as we may receive null mappings. This occurs where a query result
++ * includes a column specification which does not map to any particular real
++ * column, e.g. COUNT queries or where no-arg functions like now() are used
++ */
++public class SelectionColumnMapping implements SelectionColumns
++{
++ // Uses a LinkedHashSet as both order and uniqueness need to be preserved
++ private final LinkedHashSet<ColumnSpecification> columnSpecifications;
++ private final HashMultimap<ColumnSpecification, ColumnDefinition> columnMappings;
++
++ private SelectionColumnMapping()
++ {
++ this.columnSpecifications = new LinkedHashSet<>();
++ this.columnMappings = HashMultimap.create();
++ }
++
++ protected static SelectionColumnMapping newMapping()
++ {
++ return new SelectionColumnMapping();
++ }
++
++ protected static SelectionColumnMapping simpleMapping(List<ColumnDefinition> columnDefinitions)
++ {
++ SelectionColumnMapping mapping = new SelectionColumnMapping();
++ for (ColumnDefinition def: columnDefinitions)
++ mapping.addMapping(def, def);
++ return mapping;
++ }
++
++ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, ColumnDefinition column)
++ {
++ columnSpecifications.add(colSpec);
++ // some AbstractFunctionSelector impls do not map directly to an underlying column
++ // so don't record a mapping in that case
++ if (null != column)
++ columnMappings.put(colSpec, column);
++ return this;
++ }
++
++ public List<ColumnSpecification> getColumnSpecifications()
++ {
++ // return a mutable copy as we may add extra columns
++ // for ordering (CASSANDRA-4911 & CASSANDRA-8286)
++ return Lists.newArrayList(columnSpecifications);
++ }
++
++ public Multimap<ColumnSpecification, ColumnDefinition> getMappings()
++ {
++ return Multimaps.unmodifiableMultimap(columnMappings);
++ }
++
++ public boolean equals(Object obj)
++ {
++ if (obj == null)
++ return false;
++
++ if (!(obj instanceof SelectionColumnMapping))
++ return false;
++
++ return Objects.equal(this.columnMappings, ((SelectionColumnMapping)obj).columnMappings);
++ }
++
++ public int hashCode()
++ {
++ return Objects.hashCode(columnMappings);
++ }
++
++ public String toString()
++ {
++ final Function<ColumnDefinition, String> getDefName = new Function<ColumnDefinition, String>()
++ {
++ public String apply(ColumnDefinition def)
++ {
++ return def.name.toString();
++ }
++ };
++ final Function<ColumnSpecification, String> colSpecToMappingString = new Function<ColumnSpecification, String>()
++ {
++ public String apply(ColumnSpecification colSpec)
++ {
++ StringBuilder builder = new StringBuilder();
++ builder.append(colSpec.name.toString());
++ if (columnMappings.containsKey(colSpec))
++ {
++ builder.append(":[");
++ builder.append(Joiner.on(',').join(Iterables.transform(columnMappings.get(colSpec), getDefName)));
++ builder.append("]");
++ }
++ else
++ {
++ builder.append(":[]");
++ }
++ return builder.toString();
++ }
++ };
++
++ StringBuilder builder = new StringBuilder();
++ builder.append("{ ");
++ builder.append(Joiner.on(", ").join(Iterables.transform(columnSpecifications, colSpecToMappingString)));
++ builder.append(" }");
++ return builder.toString();
++ }
++}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
index 0000000,0000000..af334e6
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
@@@ -1,0 -1,0 +1,18 @@@
++package org.apache.cassandra.cql3.selection;
++
++import java.util.List;
++
++import com.google.common.collect.Multimap;
++
++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, ColumnDefinition> getMappings();
++}
[11/12] cassandra git commit: Merge branch 'cassandra-2.1' into
cassandra-2.2
Posted by sa...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1f8516da
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1f8516da
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1f8516da
Branch: refs/heads/trunk
Commit: 1f8516da82fc19500d3b00d37780265c34a358b0
Parents: a133992 0452e74
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 18 17:39:43 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jun 18 17:41:06 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/cassandra/cql3/ColumnIdentifier.java | 6 +-
.../cassandra/cql3/ColumnSpecification.java | 6 +
.../restrictions/StatementRestrictions.java | 22 +-
.../selection/AbstractFunctionSelector.java | 13 +
.../cassandra/cql3/selection/FieldSelector.java | 8 +-
.../cassandra/cql3/selection/Selectable.java | 10 +-
.../cassandra/cql3/selection/Selection.java | 58 +--
.../cql3/selection/SelectionColumnMapping.java | 118 +++++++
.../cql3/selection/SelectionColumns.java | 18 +
.../cassandra/cql3/selection/Selector.java | 13 +
.../cql3/selection/SelectorFactories.java | 2 +-
.../cql3/selection/SimpleSelector.java | 17 +-
.../cql3/selection/WritetimeOrTTLSelector.java | 14 +-
.../cql3/statements/SelectStatement.java | 28 +-
.../selection/SelectionColumnMappingTest.java | 353 +++++++++++++++++++
16 files changed, 633 insertions(+), 54 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 76a42ab,899ea7c..fb44c9a
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,16 -1,7 +1,17 @@@
-2.1.7
+2.2
+ * Rename class for DATE type in Java driver (CASSANDRA-9563)
+ * Duplicate compilation of UDFs on coordinator (CASSANDRA-9475)
+ * Fix connection leak in CqlRecordWriter (CASSANDRA-9576)
+ * Mlockall before opening system sstables & remove boot_without_jna option (CASSANDRA-9573)
+ * Add functions to convert timeuuid to date or time, deprecate dateOf and unixTimestampOf (CASSANDRA-9229)
+ * Make sure we cancel non-compacting sstables from LifecycleTransaction (CASSANDRA-9566)
+ * Fix deprecated repair JMX API (CASSANDRA-9570)
+ * Add logback metrics (CASSANDRA-9378)
+Merged from 2.1:
* Fix memory leak in Ref due to ConcurrentLinkedQueue.remove() behaviour (CASSANDRA-9549)
+ * Make rebuild only run one at a time (CASSANDRA-9119)
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/1f8516da/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 467b214,1501479..823af94
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@@ -17,20 -17,16 +17,20 @@@
*/
package org.apache.cassandra.cql3;
-import java.util.Locale;
+ import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Locale;
- import java.nio.ByteBuffer;
import org.apache.cassandra.cache.IMeasurableMemory;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.statements.Selectable;
+import org.apache.cassandra.cql3.selection.Selectable;
+import org.apache.cassandra.cql3.selection.Selector;
+import org.apache.cassandra.cql3.selection.SimpleSelector;
import org.apache.cassandra.db.marshal.AbstractType;
- import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.db.marshal.CompositeType;
import org.apache.cassandra.db.marshal.UTF8Type;
++import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.ObjectSizes;
import org.apache.cassandra.utils.memory.AbstractAllocator;
@@@ -109,14 -105,6 +109,14 @@@ public class ColumnIdentifier extends o
return new ColumnIdentifier(allocator.clone(bytes), text);
}
+ public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs) throws InvalidRequestException
+ {
+ ColumnDefinition def = cfm.getColumnDefinition(this);
+ if (def == null)
+ throw new InvalidRequestException(String.format("Undefined name %s in selection clause", this));
+
- return SimpleSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs), def.type);
++ return SimpleSelector.newFactory(def, addAndGetIndex(def, defs));
+ }
/**
* Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnSpecification.java
index bc5a914,f5f921d..e12a57e
--- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
@@@ -17,11 -17,9 +17,12 @@@
*/
package org.apache.cassandra.cql3;
+ import com.google.common.base.Objects;
-
import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+
+import java.util.Collection;
+import java.util.Iterator;
public class ColumnSpecification
{
@@@ -38,51 -36,23 +39,56 @@@
this.type = type;
}
- public boolean equals(Object obj)
+ /**
+ * Returns a new <code>ColumnSpecification</code> for the same column but with the specified alias.
+ *
+ * @param alias the column alias
+ * @return a new <code>ColumnSpecification</code> for the same column but with the specified alias.
+ */
+ public ColumnSpecification withAlias(ColumnIdentifier alias)
+ {
+ return new ColumnSpecification(ksName, cfName, alias, type);
+ }
+
+ public boolean isReversedType()
{
- if (null == obj)
+ return type instanceof ReversedType;
+ }
+
+ /**
+ * Returns true if all ColumnSpecifications are in the same table, false otherwise.
+ */
+ public static boolean allInSameTable(Collection<ColumnSpecification> names)
+ {
+ if (names == null || names.isEmpty())
return false;
- if(!(obj instanceof ColumnSpecification))
+ Iterator<ColumnSpecification> iter = names.iterator();
+ ColumnSpecification first = iter.next();
+ while (iter.hasNext())
+ {
+ ColumnSpecification name = iter.next();
+ if (!name.ksName.equals(first.ksName) || !name.cfName.equals(first.cfName))
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public boolean equals(Object other)
+ {
+ if (!(other 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);
+ ColumnSpecification that = (ColumnSpecification) other;
+ return this.ksName.equals(that.ksName) &&
+ this.cfName.equals(that.cfName) &&
+ this.name.equals(that.name) &&
+ this.type.equals(that.type);
}
+
+ public int hashCode()
+ {
+ return Objects.hashCode(ksName, cfName, name, type);
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index f848e2e,0000000..c10a56a
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@@ -1,572 -1,0 +1,572 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * The restrictions corresponding to the relations specified on the where-clause of CQL query.
+ */
+public final class StatementRestrictions
+{
+ /**
+ * The Column Family meta data
+ */
+ public final CFMetaData cfm;
+
+ /**
+ * Restrictions on partitioning columns
+ */
+ private PrimaryKeyRestrictions partitionKeyRestrictions;
+
+ /**
+ * Restrictions on clustering columns
+ */
+ private PrimaryKeyRestrictions clusteringColumnsRestrictions;
+
+ /**
+ * Restriction on non-primary key columns (i.e. secondary index restrictions)
+ */
+ private RestrictionSet nonPrimaryKeyRestrictions;
+
+ /**
+ * The restrictions used to build the index expressions
+ */
+ private final List<Restrictions> indexRestrictions = new ArrayList<>();
+
+ /**
+ * <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise
+ */
+ private boolean usesSecondaryIndexing;
+
+ /**
+ * Specify if the query will return a range of partition keys.
+ */
+ private boolean isKeyRange;
+
+ /**
+ * Creates a new empty <code>StatementRestrictions</code>.
+ *
+ * @param cfm the column family meta data
+ * @return a new empty <code>StatementRestrictions</code>.
+ */
+ public static StatementRestrictions empty(CFMetaData cfm)
+ {
+ return new StatementRestrictions(cfm);
+ }
+
+ private StatementRestrictions(CFMetaData cfm)
+ {
+ this.cfm = cfm;
+ this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
+ this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
+ this.nonPrimaryKeyRestrictions = new RestrictionSet();
+ }
+
+ public StatementRestrictions(CFMetaData cfm,
+ List<Relation> whereClause,
+ VariableSpecifications boundNames,
+ boolean selectsOnlyStaticColumns,
+ boolean selectACollection) throws InvalidRequestException
+ {
+ this.cfm = cfm;
+ this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
+ this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
+ this.nonPrimaryKeyRestrictions = new RestrictionSet();
+
+ /*
+ * WHERE clause. For a given entity, rules are: - EQ relation conflicts with anything else (including a 2nd EQ)
+ * - Can't have more than one LT(E) relation (resp. GT(E) relation) - IN relation are restricted to row keys
+ * (for now) and conflicts with anything else (we could allow two IN for the same entity but that doesn't seem
+ * very useful) - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value
+ * in CQL so far)
+ */
+ for (Relation relation : whereClause)
+ addRestriction(relation.toRestriction(cfm, boundNames));
+
+ ColumnFamilyStore cfs = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+ SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
+
+ boolean hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(secondaryIndexManager);
+ boolean hasQueriableIndex = hasQueriableClusteringColumnIndex
+ || partitionKeyRestrictions.hasSupportingIndex(secondaryIndexManager)
+ || nonPrimaryKeyRestrictions.hasSupportingIndex(secondaryIndexManager);
+
+ // At this point, the select statement if fully constructed, but we still have a few things to validate
+ processPartitionKeyRestrictions(hasQueriableIndex);
+
+ // Some but not all of the partition key columns have been specified;
+ // hence we need turn these restrictions into index expressions.
+ if (usesSecondaryIndexing)
+ indexRestrictions.add(partitionKeyRestrictions);
+
+ checkFalse(selectsOnlyStaticColumns && hasClusteringColumnsRestriction(),
+ "Cannot restrict clustering columns when selecting only static columns");
+
+ processClusteringColumnsRestrictions(hasQueriableIndex, selectACollection);
+
+ // Covers indexes on the first clustering column (among others).
+ if (isKeyRange && hasQueriableClusteringColumnIndex)
+ usesSecondaryIndexing = true;
+
+ usesSecondaryIndexing = usesSecondaryIndexing || clusteringColumnsRestrictions.isContains();
+
+ if (usesSecondaryIndexing)
+ indexRestrictions.add(clusteringColumnsRestrictions);
+
+ // 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 (!nonPrimaryKeyRestrictions.isEmpty())
+ {
+ usesSecondaryIndexing = true;
+ indexRestrictions.add(nonPrimaryKeyRestrictions);
+ }
+
+ if (usesSecondaryIndexing)
+ validateSecondaryIndexSelections(selectsOnlyStaticColumns);
+ }
+
+ private void addRestriction(Restriction restriction) throws InvalidRequestException
+ {
+ if (restriction.isMultiColumn())
+ clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+ else if (restriction.isOnToken())
+ partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+ else
+ addSingleColumnRestriction((SingleColumnRestriction) restriction);
+ }
+
+ public Iterable<Function> getFunctions()
+ {
+ return Iterables.concat(partitionKeyRestrictions.getFunctions(),
+ clusteringColumnsRestrictions.getFunctions(),
+ nonPrimaryKeyRestrictions.getFunctions());
+ }
+
+ private void addSingleColumnRestriction(SingleColumnRestriction restriction) throws InvalidRequestException
+ {
+ ColumnDefinition def = restriction.columnDef;
+ if (def.isPartitionKey())
+ partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+ else if (def.isClusteringColumn())
+ clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+ else
+ nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction(restriction);
+ }
+
+ /**
+ * Checks if the restrictions on the partition key is an IN restriction.
+ *
+ * @return <code>true</code> the restrictions on the partition key is an IN restriction, <code>false</code>
+ * otherwise.
+ */
+ public boolean keyIsInRelation()
+ {
+ return partitionKeyRestrictions.isIN();
+ }
+
+ /**
+ * Checks if the query request a range of partition keys.
+ *
+ * @return <code>true</code> if the query request a range of partition keys, <code>false</code> otherwise.
+ */
+ public boolean isKeyRange()
+ {
+ return this.isKeyRange;
+ }
+
+ /**
+ * Checks if the secondary index need to be queried.
+ *
+ * @return <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise.
+ */
+ public boolean usesSecondaryIndexing()
+ {
+ return this.usesSecondaryIndexing;
+ }
+
+ private void processPartitionKeyRestrictions(boolean hasQueriableIndex) 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
+ // - Is it queriable without 2ndary index, which is always more efficient
+ // If a component of the partition key is restricted by a relation, all preceding
+ // components must have a EQ. Only the last partition key component can be in IN relation.
+ if (partitionKeyRestrictions.isOnToken())
+ isKeyRange = true;
+
+ if (hasPartitionKeyUnrestrictedComponents())
+ {
+ if (!partitionKeyRestrictions.isEmpty())
+ {
+ if (!hasQueriableIndex)
+ throw invalidRequest("Partition key parts: %s must be restricted as other parts are",
+ Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents()));
+ }
+
+ isKeyRange = true;
+ usesSecondaryIndexing = hasQueriableIndex;
+ }
+ }
+
+ /**
+ * Checks if the partition key has some unrestricted components.
+ * @return <code>true</code> if the partition key has some unrestricted components, <code>false</code> otherwise.
+ */
+ private boolean hasPartitionKeyUnrestrictedComponents()
+ {
+ return partitionKeyRestrictions.size() < cfm.partitionKeyColumns().size();
+ }
+
+ /**
+ * Returns the partition key components that are not restricted.
+ * @return the partition key components that are not restricted.
+ */
+ private List<ColumnIdentifier> getPartitionKeyUnrestrictedComponents()
+ {
+ List<ColumnDefinition> list = new ArrayList<>(cfm.partitionKeyColumns());
+ list.removeAll(partitionKeyRestrictions.getColumnDefs());
+ return ColumnDefinition.toIdentifiers(list);
+ }
+
+ /**
+ * Processes the clustering column restrictions.
+ *
+ * @param hasQueriableIndex <code>true</code> if some of the queried data are indexed, <code>false</code> otherwise
+ * @param selectACollection <code>true</code> if the query should return a collection column
+ * @throws InvalidRequestException if the request is invalid
+ */
+ private void processClusteringColumnsRestrictions(boolean hasQueriableIndex,
+ boolean selectACollection) throws InvalidRequestException
+ {
+ checkFalse(clusteringColumnsRestrictions.isIN() && selectACollection,
+ "Cannot restrict clustering columns by IN relations when a collection is selected by the query");
+ checkFalse(clusteringColumnsRestrictions.isContains() && !hasQueriableIndex,
+ "Cannot restrict clustering columns by a CONTAINS relation without a secondary index");
+
+ if (hasClusteringColumnsRestriction())
+ {
+ List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
+ List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
+
+ for (int i = 0, m = restrictedColumns.size(); i < m; i++)
+ {
+ ColumnDefinition clusteringColumn = clusteringColumns.get(i);
+ ColumnDefinition restrictedColumn = restrictedColumns.get(i);
+
+ if (!clusteringColumn.equals(restrictedColumn))
+ {
+ checkTrue(hasQueriableIndex,
+ "PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted",
+ restrictedColumn.name,
+ clusteringColumn.name);
+
+ usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
+ break;
+ }
+ }
+ }
+
+ if (clusteringColumnsRestrictions.isContains())
+ usesSecondaryIndexing = true;
+ }
+
+ public List<IndexExpression> getIndexExpressions(SecondaryIndexManager indexManager,
+ QueryOptions options) throws InvalidRequestException
+ {
+ if (!usesSecondaryIndexing || indexRestrictions.isEmpty())
+ return Collections.emptyList();
+
+ List<IndexExpression> expressions = new ArrayList<>();
+ for (Restrictions restrictions : indexRestrictions)
+ restrictions.addIndexExpressionTo(expressions, indexManager, options);
+
+ return expressions;
+ }
+
+ /**
+ * Returns the partition keys for which the data is requested.
+ *
+ * @param options the query options
+ * @return the partition keys for which the data is requested.
+ * @throws InvalidRequestException if the partition keys cannot be retrieved
+ */
+ public Collection<ByteBuffer> getPartitionKeys(final QueryOptions options) throws InvalidRequestException
+ {
+ return partitionKeyRestrictions.values(options);
+ }
+
+ /**
+ * Returns the specified bound of the partition key.
+ *
+ * @param b the boundary type
+ * @param options the query options
+ * @return the specified bound of the partition key
+ * @throws InvalidRequestException if the boundary cannot be retrieved
+ */
+ private ByteBuffer getPartitionKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
+ {
+ // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the
+ // first
+ // component of a composite partition key).
+ if (hasPartitionKeyUnrestrictedComponents())
+ return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+ // We deal with IN queries for keys in other places, so we know buildBound will return only one result
+ return partitionKeyRestrictions.bounds(b, options).get(0);
+ }
+
+ /**
+ * Returns the partition key bounds.
+ *
+ * @param options the query options
+ * @return the partition key bounds
+ * @throws InvalidRequestException if the query is invalid
+ */
+ public AbstractBounds<RowPosition> getPartitionKeyBounds(QueryOptions options) throws InvalidRequestException
+ {
+ IPartitioner p = StorageService.getPartitioner();
+
+ if (partitionKeyRestrictions.isOnToken())
+ {
+ return getPartitionKeyBoundsForTokenRestrictions(p, options);
+ }
+
+ return getPartitionKeyBounds(p, options);
+ }
+
+ private AbstractBounds<RowPosition> getPartitionKeyBounds(IPartitioner p,
+ QueryOptions options) throws InvalidRequestException
+ {
+ ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options);
+ ByteBuffer finishKeyBytes = getPartitionKeyBound(Bound.END, options);
+
+ RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
+ RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
+
+ if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum())
+ return null;
+
+ if (partitionKeyRestrictions.isInclusive(Bound.START))
+ {
+ return partitionKeyRestrictions.isInclusive(Bound.END)
+ ? new Bounds<>(startKey, finishKey)
+ : new IncludingExcludingBounds<>(startKey, finishKey);
+ }
+
+ return partitionKeyRestrictions.isInclusive(Bound.END)
+ ? new Range<>(startKey, finishKey)
+ : new ExcludingBounds<>(startKey, finishKey);
+ }
+
+ private AbstractBounds<RowPosition> getPartitionKeyBoundsForTokenRestrictions(IPartitioner p,
+ QueryOptions options)
+ throws InvalidRequestException
+ {
+ Token startToken = getTokenBound(Bound.START, options, p);
+ Token endToken = getTokenBound(Bound.END, options, p);
+
+ boolean includeStart = partitionKeyRestrictions.isInclusive(Bound.START);
+ boolean includeEnd = partitionKeyRestrictions.isInclusive(Bound.END);
+
+ /*
+ * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
+ * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result in that
+ * case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
+ *
+ * In practice, we want to return an empty result set if either startToken > endToken, or both are equal but
+ * one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a) or (a, a)).
+ * Note though that in the case where startToken or endToken is the minimum token, then this special case
+ * rule should not apply.
+ */
+ int cmp = startToken.compareTo(endToken);
+ if (!startToken.isMinimum() && !endToken.isMinimum()
+ && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
+ return null;
+
+ RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
+ RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
+
+ return new Range<>(start, end);
+ }
+
+ private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
+ {
+ if (!partitionKeyRestrictions.hasBound(b))
+ return p.getMinimumToken();
+
+ ByteBuffer value = partitionKeyRestrictions.bounds(b, options).get(0);
+ checkNotNull(value, "Invalid null token value");
+ return p.getTokenFactory().fromByteArray(value);
+ }
+
+ /**
+ * Checks if the query does not contains any restriction on the clustering columns.
+ *
+ * @return <code>true</code> if the query does not contains any restriction on the clustering columns,
+ * <code>false</code> otherwise.
+ */
+ public boolean hasNoClusteringColumnsRestriction()
+ {
+ return clusteringColumnsRestrictions.isEmpty();
+ }
+
++ /**
++ * Checks if the query has some restrictions on the clustering columns.
++ *
++ * @return <code>true</code> if the query has some restrictions on the clustering columns,
++ * <code>false</code> otherwise.
++ */
++ public boolean hasClusteringColumnsRestriction()
++ {
++ return !clusteringColumnsRestrictions.isEmpty();
++ }
++
+ // For non-composite slices, we don't support internally the difference between exclusive and
+ // inclusive bounds, so we deal with it manually.
+ public boolean isNonCompositeSliceWithExclusiveBounds()
+ {
+ return !cfm.comparator.isCompound()
+ && clusteringColumnsRestrictions.isSlice()
+ && (!clusteringColumnsRestrictions.isInclusive(Bound.START) || !clusteringColumnsRestrictions.isInclusive(Bound.END));
+ }
+
+ /**
+ * Returns the requested clustering columns as <code>Composite</code>s.
+ *
+ * @param options the query options
+ * @return the requested clustering columns as <code>Composite</code>s
+ * @throws InvalidRequestException if the query is not valid
+ */
+ public List<Composite> getClusteringColumnsAsComposites(QueryOptions options) throws InvalidRequestException
+ {
+ return clusteringColumnsRestrictions.valuesAsComposites(options);
+ }
+
+ /**
+ * Returns the bounds (start or end) of the clustering columns as <code>Composites</code>.
+ *
+ * @param b the bound type
+ * @param options the query options
+ * @return the bounds (start or end) of the clustering columns as <code>Composites</code>
+ * @throws InvalidRequestException if the request is not valid
+ */
+ public List<Composite> getClusteringColumnsBoundsAsComposites(Bound b,
+ QueryOptions options) throws InvalidRequestException
+ {
+ return clusteringColumnsRestrictions.boundsAsComposites(b, options);
+ }
+
+ /**
+ * Returns the bounds (start or end) of the clustering columns.
+ *
+ * @param b the bound type
+ * @param options the query options
+ * @return the bounds (start or end) of the clustering columns
+ * @throws InvalidRequestException if the request is not valid
+ */
+ public List<ByteBuffer> getClusteringColumnsBounds(Bound b, QueryOptions options) throws InvalidRequestException
+ {
+ return clusteringColumnsRestrictions.bounds(b, options);
+ }
+
+ /**
+ * Checks if the bounds (start or end) of the clustering columns are inclusive.
+ *
+ * @param bound the bound type
+ * @return <code>true</code> if the bounds (start or end) of the clustering columns are inclusive,
+ * <code>false</code> otherwise
+ */
+ public boolean areRequestedBoundsInclusive(Bound bound)
+ {
+ return clusteringColumnsRestrictions.isInclusive(bound);
+ }
+
+ /**
+ * Checks if the query returns a range of columns.
+ *
+ * @return <code>true</code> if the query returns a range of columns, <code>false</code> otherwise.
+ */
+ public boolean isColumnRange()
+ {
+ // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
+ // Static CF (non dense but non composite) never entails a column slice however
+ if (!cfm.comparator.isDense())
+ return cfm.comparator.isCompound();
+
+ // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about
+ // CASSANDRA-5762),
+ // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
+ return clusteringColumnsRestrictions.size() < cfm.clusteringColumns().size() || clusteringColumnsRestrictions.isSlice();
+ }
+
+ /**
+ * Checks if the query need to use filtering.
+ * @return <code>true</code> if the query need to use filtering, <code>false</code> otherwise.
+ */
+ public boolean needFiltering()
+ {
+ int numberOfRestrictedColumns = 0;
+ for (Restrictions restrictions : indexRestrictions)
+ numberOfRestrictedColumns += restrictions.size();
+
+ return numberOfRestrictedColumns > 1
+ || (numberOfRestrictedColumns == 0 && !clusteringColumnsRestrictions.isEmpty())
+ || (numberOfRestrictedColumns != 0
+ && nonPrimaryKeyRestrictions.hasMultipleContains());
+ }
+
+ private void validateSecondaryIndexSelections(boolean selectsOnlyStaticColumns) throws InvalidRequestException
+ {
+ checkFalse(keyIsInRelation(),
+ "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.
+ checkFalse(selectsOnlyStaticColumns, "Queries using 2ndary indexes don't support selecting only static columns");
+ }
+
- /**
- * Checks if the query has some restrictions on the clustering columns.
- *
- * @return <code>true</code> if the query has some restrictions on the clustering columns,
- * <code>false</code> otherwise.
- */
- private boolean hasClusteringColumnsRestriction()
- {
- return !clusteringColumnsRestrictions.isEmpty();
- }
-
+ public void reverse()
+ {
+ clusteringColumnsRestrictions = new ReversedPrimaryKeyRestrictions(clusteringColumnsRestrictions);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index 1dd1903,0000000..fa40152
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@@ -1,120 -1,0 +1,133 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import com.google.common.collect.Iterables;
+import org.apache.commons.lang3.text.StrBuilder;
+
++import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+abstract class AbstractFunctionSelector<T extends Function> extends Selector
+{
+ protected final T fun;
+
+ /**
+ * The list used to pass the function arguments is recycled to avoid the cost of instantiating a new list
+ * with each function call.
+ */
+ protected final List<ByteBuffer> args;
+ protected final List<Selector> argSelectors;
+
+ public static Factory newFactory(final Function fun, final SelectorFactories factories) throws InvalidRequestException
+ {
+ if (fun.isAggregate())
+ {
+ if (factories.doesAggregation())
+ throw new InvalidRequestException("aggregate functions cannot be used as arguments of aggregate functions");
+ }
+ else
+ {
+ if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
+ throw new InvalidRequestException(String.format("arguments of function %s must be either all aggregates or no aggregates",
+ fun.name()));
+ }
+
+ return new Factory()
+ {
+ protected String getColumnName()
+ {
+ return new StrBuilder(fun.name().toString()).append('(')
+ .appendWithSeparators(factories.getColumnNames(), ", ")
+ .append(')')
+ .toString();
+ }
+
+ protected AbstractType<?> getReturnType()
+ {
+ return fun.returnType();
+ }
+
++ protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
++ {
++ for (Factory factory : factories)
++ factory.addColumnMapping(mapping, resultsColumn);
++
++ if (mapping.getMappings().get(resultsColumn).isEmpty())
++ // add a null mapping for cases where there are no
++ // further selectors, such as no-arg functions and count
++ mapping.addMapping(resultsColumn, null);
++
++ }
++
+ public Iterable<Function> getFunctions()
+ {
+ return Iterables.concat(fun.getFunctions(), factories.getFunctions());
+ }
+
+ public Selector newInstance() throws InvalidRequestException
+ {
+ return fun.isAggregate() ? new AggregateFunctionSelector(fun, factories.newInstances())
+ : new ScalarFunctionSelector(fun, factories.newInstances());
+ }
+
+ public boolean isWritetimeSelectorFactory()
+ {
+ return factories.containsWritetimeSelectorFactory();
+ }
+
+ public boolean isTTLSelectorFactory()
+ {
+ return factories.containsTTLSelectorFactory();
+ }
+
+ public boolean isAggregateSelectorFactory()
+ {
+ return fun.isAggregate() || factories.containsOnlyAggregateFunctions();
+ }
+ };
+ }
+
+ protected AbstractFunctionSelector(T fun, List<Selector> argSelectors)
+ {
+ this.fun = fun;
+ this.argSelectors = argSelectors;
+ this.args = Arrays.asList(new ByteBuffer[argSelectors.size()]);
+ }
+
+ public AbstractType<?> getType()
+ {
+ return fun.returnType();
+ }
+
+ @Override
+ public String toString()
+ {
+ return new StrBuilder().append(fun.name())
+ .append("(")
+ .appendWithSeparators(argSelectors, ", ")
+ .append(")")
+ .toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
index 76dbb22,0000000..63b6cc6
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
@@@ -1,103 -1,0 +1,109 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+
++import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+final class FieldSelector extends Selector
+{
+ private final UserType type;
+ private final int field;
+ private final Selector selected;
+
+ public static Factory newFactory(final UserType type, final int field, final Selector.Factory factory)
+ {
+ return new Factory()
+ {
+ protected String getColumnName()
+ {
+ return String.format("%s.%s",
+ factory.getColumnName(),
+ UTF8Type.instance.getString(type.fieldName(field)));
+ }
+
+ protected AbstractType<?> getReturnType()
+ {
+ return type.fieldType(field);
+ }
+
++ protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
++ {
++ factory.addColumnMapping(mapping, resultsColumn);
++ }
++
+ public Selector newInstance() throws InvalidRequestException
+ {
+ return new FieldSelector(type, field, factory.newInstance());
+ }
+
+ public boolean isAggregateSelectorFactory()
+ {
+ return factory.isAggregateSelectorFactory();
+ }
+ };
+ }
+
+ public boolean isAggregate()
+ {
+ return false;
+ }
+
+ public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+ {
+ selected.addInput(protocolVersion, rs);
+ }
+
+ public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+ {
+ ByteBuffer value = selected.getOutput(protocolVersion);
+ if (value == null)
+ return null;
+ ByteBuffer[] buffers = type.split(value);
+ return field < buffers.length ? buffers[field] : null;
+ }
+
+ public AbstractType<?> getType()
+ {
+ return type.fieldType(field);
+ }
+
+ public void reset()
+ {
+ selected.reset();
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("%s.%s", selected, UTF8Type.instance.getString(type.fieldName(field)));
+ }
+
+ private FieldSelector(UserType type, int field, Selector selected)
+ {
+ this.type = type;
+ this.field = field;
+ this.selected = selected;
+ }
- }
++}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 4506111,0000000..ee134ee
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@@ -1,253 -1,0 +1,251 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.util.ArrayList;
+import java.util.List;
+
++import org.apache.commons.lang3.text.StrBuilder;
++
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
- import org.apache.cassandra.cql3.functions.Function;
- import org.apache.cassandra.cql3.functions.FunctionName;
- import org.apache.cassandra.cql3.functions.Functions;
- import org.apache.cassandra.cql3.functions.ToJsonFct;
++import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
- import org.apache.commons.lang3.text.StrBuilder;
+
+public abstract class Selectable
+{
+ public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
+ throws InvalidRequestException;
+
+ protected static int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
+ {
+ int idx = l.indexOf(def);
+ if (idx < 0)
+ {
+ idx = l.size();
+ l.add(def);
+ }
+ return idx;
+ }
+
+ public static interface Raw
+ {
+ public Selectable prepare(CFMetaData cfm);
+
+ /**
+ * Returns true if any processing is performed on the selected column.
+ **/
+ public boolean processesSelection();
+ }
+
+ public static class WritetimeOrTTL extends Selectable
+ {
+ public final ColumnIdentifier id;
+ public final boolean isWritetime;
+
+ public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
+ {
+ this.id = id;
+ this.isWritetime = isWritetime;
+ }
+
+ @Override
+ public String toString()
+ {
+ return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
+ }
+
+ public Selector.Factory newSelectorFactory(CFMetaData cfm,
+ List<ColumnDefinition> defs) throws InvalidRequestException
+ {
+ ColumnDefinition def = cfm.getColumnDefinition(id);
+ if (def == null)
+ throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
+ if (def.isPrimaryKeyColumn())
+ throw new InvalidRequestException(
+ String.format("Cannot use selection function %s on PRIMARY KEY part %s",
+ isWritetime ? "writeTime" : "ttl",
+ def.name));
+ if (def.type.isCollection())
+ throw new InvalidRequestException(String.format("Cannot use selection function %s on collections",
+ isWritetime ? "writeTime" : "ttl"));
+
- return WritetimeOrTTLSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs), isWritetime);
++ return WritetimeOrTTLSelector.newFactory(def, addAndGetIndex(def, defs), isWritetime);
+ }
+
+ public static class Raw implements Selectable.Raw
+ {
+ private final ColumnIdentifier.Raw id;
+ private final boolean isWritetime;
+
+ public Raw(ColumnIdentifier.Raw id, boolean isWritetime)
+ {
+ this.id = id;
+ this.isWritetime = isWritetime;
+ }
+
+ public WritetimeOrTTL prepare(CFMetaData cfm)
+ {
+ return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
+ }
+
+ public boolean processesSelection()
+ {
+ return true;
+ }
+ }
+ }
+
+ public static class WithFunction extends Selectable
+ {
+ public final FunctionName functionName;
+ public final List<Selectable> args;
+
+ public WithFunction(FunctionName functionName, List<Selectable> args)
+ {
+ this.functionName = functionName;
+ this.args = args;
+ }
+
+ @Override
+ public String toString()
+ {
+ return new StrBuilder().append(functionName)
+ .append("(")
+ .appendWithSeparators(args, ", ")
+ .append(")")
+ .toString();
+ }
+
+ public Selector.Factory newSelectorFactory(CFMetaData cfm,
+ List<ColumnDefinition> defs) throws InvalidRequestException
+ {
+ SelectorFactories factories =
+ SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, cfm, defs);
+
+ // We need to circumvent the normal function lookup process for toJson() because instances of the function
+ // are not pre-declared (because it can accept any type of argument).
+ Function fun;
+ if (functionName.equalsNativeFunction(ToJsonFct.NAME))
+ fun = ToJsonFct.getInstance(factories.getReturnTypes());
+ else
+ fun = Functions.get(cfm.ksName, functionName, factories.newInstances(), cfm.ksName, cfm.cfName, null);
+
+ if (fun == null)
+ throw new InvalidRequestException(String.format("Unknown function '%s'", functionName));
+ if (fun.returnType() == null)
+ throw new InvalidRequestException(String.format("Unknown function %s called in selection clause",
+ functionName));
+
+ return AbstractFunctionSelector.newFactory(fun, factories);
+ }
+
+ public static class Raw implements Selectable.Raw
+ {
+ private final FunctionName functionName;
+ private final List<Selectable.Raw> args;
+
+ public Raw(FunctionName functionName, List<Selectable.Raw> args)
+ {
+ this.functionName = functionName;
+ this.args = args;
+ }
+
+ public WithFunction prepare(CFMetaData cfm)
+ {
+ List<Selectable> preparedArgs = new ArrayList<>(args.size());
+ for (Selectable.Raw arg : args)
+ preparedArgs.add(arg.prepare(cfm));
+ return new WithFunction(functionName, preparedArgs);
+ }
+
+ public boolean processesSelection()
+ {
+ return true;
+ }
+ }
+ }
+
+ public static class WithFieldSelection extends Selectable
+ {
+ public final Selectable selected;
+ public final ColumnIdentifier field;
+
+ public WithFieldSelection(Selectable selected, ColumnIdentifier field)
+ {
+ this.selected = selected;
+ this.field = field;
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("%s.%s", selected, field);
+ }
+
+ public Selector.Factory newSelectorFactory(CFMetaData cfm,
+ List<ColumnDefinition> defs) throws InvalidRequestException
+ {
+ Selector.Factory factory = selected.newSelectorFactory(cfm, defs);
+ AbstractType<?> type = factory.newInstance().getType();
+ if (!(type instanceof UserType))
+ throw new InvalidRequestException(
+ String.format("Invalid field selection: %s of type %s is not a user type",
+ selected,
+ type.asCQL3Type()));
+
+ UserType ut = (UserType) type;
+ for (int i = 0; i < ut.size(); i++)
+ {
+ if (!ut.fieldName(i).equals(field.bytes))
+ continue;
+ return FieldSelector.newFactory(ut, i, factory);
+ }
+ throw new InvalidRequestException(String.format("%s of type %s has no field %s",
+ selected,
+ type.asCQL3Type(),
+ field));
+ }
+
+ public static class Raw implements Selectable.Raw
+ {
+ private final Selectable.Raw selected;
+ private final ColumnIdentifier.Raw field;
+
+ public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field)
+ {
+ this.selected = selected;
+ this.field = field;
+ }
+
+ public WithFieldSelection prepare(CFMetaData cfm)
+ {
+ return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm));
+ }
+
+ public boolean processesSelection()
+ {
+ return true;
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selection.java
index 9c990ce,0000000..25278df
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@@ -1,535 -1,0 +1,547 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.CounterCell;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public abstract class Selection
+{
+ /**
+ * A predicate that returns <code>true</code> for static columns.
+ */
+ private static final Predicate<ColumnDefinition> STATIC_COLUMN_FILTER = new Predicate<ColumnDefinition>()
+ {
+ public boolean apply(ColumnDefinition def)
+ {
+ return def.isStatic();
+ }
+ };
+
+ private final CFMetaData cfm;
- private final Collection<ColumnDefinition> columns;
++ private final List<ColumnDefinition> columns;
++ private final SelectionColumnMapping columnMapping;
+ private final ResultSet.ResultMetadata metadata;
+ private final boolean collectTimestamps;
+ private final boolean collectTTLs;
+
+ protected Selection(CFMetaData cfm,
- Collection<ColumnDefinition> columns,
- List<ColumnSpecification> metadata,
++ List<ColumnDefinition> columns,
++ SelectionColumnMapping columnMapping,
+ boolean collectTimestamps,
+ boolean collectTTLs)
+ {
+ this.cfm = cfm;
+ this.columns = columns;
- this.metadata = new ResultSet.ResultMetadata(metadata);
++ this.columnMapping = columnMapping;
++ this.metadata = new ResultSet.ResultMetadata(columnMapping.getColumnSpecifications());
+ this.collectTimestamps = collectTimestamps;
+ this.collectTTLs = collectTTLs;
+ }
+
+ // Overriden by SimpleSelection when appropriate.
+ public boolean isWildcard()
+ {
+ return false;
+ }
+
+ /**
+ * Checks if this selection contains static columns.
+ * @return <code>true</code> if this selection contains static columns, <code>false</code> otherwise;
+ */
+ public boolean containsStaticColumns()
+ {
+ if (!cfm.hasStaticColumns())
+ return false;
+
+ if (isWildcard())
+ return true;
+
+ return !Iterables.isEmpty(Iterables.filter(columns, STATIC_COLUMN_FILTER));
+ }
+
+ /**
+ * Checks if this selection contains only static columns.
+ * @return <code>true</code> if this selection contains only static columns, <code>false</code> otherwise;
+ */
+ public boolean containsOnlyStaticColumns()
+ {
+ if (!containsStaticColumns())
+ return false;
+
+ if (isWildcard())
+ return false;
+
+ for (ColumnDefinition def : getColumns())
+ {
+ if (!def.isPartitionKey() && !def.isStatic())
+ return false;
+ }
+
+ return true;
+ }
+
+ /**
+ * Checks if this selection contains a collection.
+ *
+ * @return <code>true</code> if this selection contains a collection, <code>false</code> otherwise.
+ */
+ public boolean containsACollection()
+ {
+ if (!cfm.comparator.hasCollections())
+ return false;
+
+ for (ColumnDefinition def : getColumns())
+ if (def.type.isCollection() && def.type.isMultiCell())
+ return true;
+
+ return false;
+ }
+
+ /**
+ * Returns the index of the specified column.
+ *
+ * @param def the column definition
+ * @return the index of the specified column
+ */
+ public int indexOf(final ColumnDefinition def)
+ {
+ return Iterators.indexOf(getColumns().iterator(), new Predicate<ColumnDefinition>()
+ {
+ public boolean apply(ColumnDefinition n)
+ {
+ return def.name.equals(n.name);
+ }
+ });
+ }
+
+ public ResultSet.ResultMetadata getResultMetadata(boolean isJson)
+ {
+ if (!isJson)
+ return metadata;
+
+ ColumnSpecification firstColumn = metadata.names.get(0);
+ ColumnSpecification jsonSpec = new ColumnSpecification(firstColumn.ksName, firstColumn.cfName, Json.JSON_COLUMN_ID, UTF8Type.instance);
+ return new ResultSet.ResultMetadata(Arrays.asList(jsonSpec));
+ }
+
+ public static Selection wildcard(CFMetaData cfm)
+ {
- List<ColumnDefinition> all = new ArrayList<ColumnDefinition>(cfm.allColumns().size());
++ List<ColumnDefinition> all = new ArrayList<>(cfm.allColumns().size());
+ Iterators.addAll(all, cfm.allColumnsInSelectOrder());
+ return new SimpleSelection(cfm, all, true);
+ }
+
- public static Selection forColumns(CFMetaData cfm, Collection<ColumnDefinition> columns)
++ public static Selection forColumns(CFMetaData cfm, List<ColumnDefinition> columns)
+ {
+ return new SimpleSelection(cfm, columns, false);
+ }
+
+ public int addColumnForOrdering(ColumnDefinition c)
+ {
+ columns.add(c);
+ metadata.addNonSerializedColumn(c);
+ return columns.size() - 1;
+ }
+
+ public Iterable<Function> getFunctions()
+ {
+ return Collections.emptySet();
+ }
+
+ private static boolean processesSelection(List<RawSelector> rawSelectors)
+ {
+ for (RawSelector rawSelector : rawSelectors)
+ {
+ if (rawSelector.processesSelection())
+ return true;
+ }
+ return false;
+ }
+
+ public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors) throws InvalidRequestException
+ {
- List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>();
++ List<ColumnDefinition> defs = new ArrayList<>();
+
+ SelectorFactories factories =
+ SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), cfm, defs);
- List<ColumnSpecification> metadata = collectMetadata(cfm, rawSelectors, factories);
++ SelectionColumnMapping mapping = collectColumnMappings(cfm, rawSelectors, factories);
+
- return processesSelection(rawSelectors) ? new SelectionWithProcessing(cfm, defs, metadata, factories)
- : new SimpleSelection(cfm, defs, metadata, false);
++ return processesSelection(rawSelectors) ? new SelectionWithProcessing(cfm, defs, mapping, factories)
++ : new SimpleSelection(cfm, defs, mapping, false);
+ }
+
- private static List<ColumnSpecification> collectMetadata(CFMetaData cfm,
- List<RawSelector> rawSelectors,
- SelectorFactories factories)
++ private static SelectionColumnMapping collectColumnMappings(CFMetaData cfm,
++ List<RawSelector> rawSelectors,
++ SelectorFactories factories)
+ {
- List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
++ SelectionColumnMapping selectionColumns = SelectionColumnMapping.newMapping();
+ Iterator<RawSelector> iter = rawSelectors.iterator();
+ for (Selector.Factory factory : factories)
+ {
+ ColumnSpecification colSpec = factory.getColumnSpecification(cfm);
+ ColumnIdentifier alias = iter.next().alias;
- metadata.add(alias == null ? colSpec : colSpec.withAlias(alias));
++ factory.addColumnMapping(selectionColumns,
++ alias == null ? colSpec : colSpec.withAlias(alias));
+ }
- return metadata;
++ return selectionColumns;
+ }
+
+ protected abstract Selectors newSelectors() throws InvalidRequestException;
+
+ /**
+ * @return the list of CQL3 columns value this SelectionClause needs.
+ */
- public Collection<ColumnDefinition> getColumns()
++ public List<ColumnDefinition> getColumns()
+ {
+ return columns;
+ }
+
++ /**
++ * @return the mappings between resultset columns and the underlying columns
++ */
++ public SelectionColumns getColumnMapping()
++ {
++ return columnMapping;
++ }
++
+ public ResultSetBuilder resultSetBuilder(long now, boolean isJson) throws InvalidRequestException
+ {
+ return new ResultSetBuilder(now, isJson);
+ }
+
+ public abstract boolean isAggregate();
+
+ @Override
+ public String toString()
+ {
+ return Objects.toStringHelper(this)
+ .add("columns", columns)
++ .add("columnMapping", columnMapping)
+ .add("metadata", metadata)
+ .add("collectTimestamps", collectTimestamps)
+ .add("collectTTLs", collectTTLs)
+ .toString();
+ }
+
+ public class ResultSetBuilder
+ {
+ private final ResultSet resultSet;
+
+ /**
+ * As multiple thread can access a <code>Selection</code> instance each <code>ResultSetBuilder</code> will use
+ * its own <code>Selectors</code> instance.
+ */
+ private final Selectors selectors;
+
+ /*
+ * We'll build CQL3 row one by one.
+ * The currentRow is the values for the (CQL3) columns we've fetched.
+ * We also collect timestamps and ttls for the case where the writetime and
+ * ttl functions are used. Note that we might collect timestamp and/or ttls
+ * we don't care about, but since the array below are allocated just once,
+ * it doesn't matter performance wise.
+ */
+ List<ByteBuffer> current;
+ final long[] timestamps;
+ final int[] ttls;
+ final long now;
+
+ private final boolean isJson;
+
+ private ResultSetBuilder(long now, boolean isJson) throws InvalidRequestException
+ {
+ this.resultSet = new ResultSet(getResultMetadata(isJson).copy(), new ArrayList<List<ByteBuffer>>());
+ this.selectors = newSelectors();
+ this.timestamps = collectTimestamps ? new long[columns.size()] : null;
+ this.ttls = collectTTLs ? new int[columns.size()] : null;
+ this.now = now;
+ this.isJson = isJson;
+ }
+
+ public void add(ByteBuffer v)
+ {
+ current.add(v);
+ }
+
+ public void add(Cell c)
+ {
+ current.add(isDead(c) ? null : value(c));
+ if (timestamps != null)
+ {
+ timestamps[current.size() - 1] = isDead(c) ? Long.MIN_VALUE : c.timestamp();
+ }
+ if (ttls != null)
+ {
+ int ttl = -1;
+ if (!isDead(c) && c instanceof ExpiringCell)
+ ttl = c.getLocalDeletionTime() - (int) (now / 1000);
+ ttls[current.size() - 1] = ttl;
+ }
+ }
+
+ private boolean isDead(Cell c)
+ {
+ return c == null || !c.isLive(now);
+ }
+
+ public void newRow(int protocolVersion) throws InvalidRequestException
+ {
+ if (current != null)
+ {
+ selectors.addInputRow(protocolVersion, this);
+ if (!selectors.isAggregate())
+ {
+ resultSet.addRow(getOutputRow(protocolVersion));
+ selectors.reset();
+ }
+ }
+ current = new ArrayList<>(columns.size());
+ }
+
+ public ResultSet build(int protocolVersion) throws InvalidRequestException
+ {
+ if (current != null)
+ {
+ selectors.addInputRow(protocolVersion, this);
+ resultSet.addRow(getOutputRow(protocolVersion));
+ selectors.reset();
+ current = null;
+ }
+
+ if (resultSet.isEmpty() && selectors.isAggregate())
+ resultSet.addRow(getOutputRow(protocolVersion));
+ return resultSet;
+ }
+
+ private List<ByteBuffer> getOutputRow(int protocolVersion)
+ {
+ List<ByteBuffer> outputRow = selectors.getOutputRow(protocolVersion);
+ return isJson ? rowToJson(outputRow, protocolVersion)
+ : outputRow;
+ }
+
+ private List<ByteBuffer> rowToJson(List<ByteBuffer> row, int protocolVersion)
+ {
+ StringBuilder sb = new StringBuilder("{");
+ for (int i = 0; i < metadata.names.size(); i++)
+ {
+ if (i > 0)
+ sb.append(", ");
+
+ ColumnSpecification spec = metadata.names.get(i);
+ String columnName = spec.name.toString();
+ if (!columnName.equals(columnName.toLowerCase(Locale.US)))
+ columnName = "\"" + columnName + "\"";
+
+ ByteBuffer buffer = row.get(i);
+ sb.append('"');
+ sb.append(Json.JSON_STRING_ENCODER.quoteAsString(columnName));
+ sb.append("\": ");
+ if (buffer == null)
+ sb.append("null");
+ else
+ sb.append(spec.type.toJSONString(buffer, protocolVersion));
+ }
+ sb.append("}");
+ return Collections.singletonList(UTF8Type.instance.getSerializer().serialize(sb.toString()));
+ }
+
+ private ByteBuffer value(Cell c)
+ {
+ return (c instanceof CounterCell)
+ ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
+ : c.value();
+ }
+ }
+
+ private static interface Selectors
+ {
+ public boolean isAggregate();
+
+ /**
+ * Adds the current row of the specified <code>ResultSetBuilder</code>.
+ *
+ * @param rs the <code>ResultSetBuilder</code>
+ * @throws InvalidRequestException
+ */
+ public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException;
+
+ public List<ByteBuffer> getOutputRow(int protocolVersion) throws InvalidRequestException;
+
+ public void reset();
+ }
+
+ // Special cased selection for when no function is used (this save some allocations).
+ private static class SimpleSelection extends Selection
+ {
+ private final boolean isWildcard;
+
- public SimpleSelection(CFMetaData cfm, Collection<ColumnDefinition> columns, boolean isWildcard)
++ public SimpleSelection(CFMetaData cfm, List<ColumnDefinition> columns, boolean isWildcard)
+ {
- this(cfm, columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
++ this(cfm, columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
+ }
+
+ public SimpleSelection(CFMetaData cfm,
- Collection<ColumnDefinition> columns,
- List<ColumnSpecification> metadata,
++ List<ColumnDefinition> columns,
++ SelectionColumnMapping metadata,
+ boolean isWildcard)
+ {
+ /*
+ * In theory, even a simple selection could have multiple time the same column, so we
+ * could filter those duplicate out of columns. But since we're very unlikely to
+ * get much duplicate in practice, it's more efficient not to bother.
+ */
+ super(cfm, columns, metadata, false, false);
+ this.isWildcard = isWildcard;
+ }
+
+ @Override
+ public boolean isWildcard()
+ {
+ return isWildcard;
+ }
+
+ public boolean isAggregate()
+ {
+ return false;
+ }
+
+ protected Selectors newSelectors()
+ {
+ return new Selectors()
+ {
+ private List<ByteBuffer> current;
+
+ public void reset()
+ {
+ current = null;
+ }
+
+ public List<ByteBuffer> getOutputRow(int protocolVersion)
+ {
+ return current;
+ }
+
+ public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+ {
+ current = rs.current;
+ }
+
+ public boolean isAggregate()
+ {
+ return false;
+ }
+ };
+ }
+ }
+
+ private static class SelectionWithProcessing extends Selection
+ {
+ private final SelectorFactories factories;
+
+ public SelectionWithProcessing(CFMetaData cfm,
- Collection<ColumnDefinition> columns,
- List<ColumnSpecification> metadata,
++ List<ColumnDefinition> columns,
++ SelectionColumnMapping metadata,
+ SelectorFactories factories) throws InvalidRequestException
+ {
+ super(cfm,
+ columns,
+ metadata,
+ factories.containsWritetimeSelectorFactory(),
+ factories.containsTTLSelectorFactory());
+
+ this.factories = factories;
+
+ if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
+ throw new InvalidRequestException("the select clause must either contain only aggregates or no aggregate");
+ }
+
+ @Override
+ public Iterable<Function> getFunctions()
+ {
+ return factories.getFunctions();
+ }
+
+ @Override
+ public int addColumnForOrdering(ColumnDefinition c)
+ {
+ int index = super.addColumnForOrdering(c);
+ factories.addSelectorForOrdering(c, index);
+ return index;
+ }
+
+ public boolean isAggregate()
+ {
+ return factories.containsOnlyAggregateFunctions();
+ }
+
+ protected Selectors newSelectors() throws InvalidRequestException
+ {
+ return new Selectors()
+ {
+ private final List<Selector> selectors = factories.newInstances();
+
+ public void reset()
+ {
+ for (Selector selector : selectors)
+ selector.reset();
+ }
+
+ public boolean isAggregate()
+ {
+ return factories.containsOnlyAggregateFunctions();
+ }
+
+ public List<ByteBuffer> getOutputRow(int protocolVersion) throws InvalidRequestException
+ {
+ List<ByteBuffer> outputRow = new ArrayList<>(selectors.size());
+
+ for (Selector selector: selectors)
+ outputRow.add(selector.getOutput(protocolVersion));
+
+ return outputRow;
+ }
+
+ public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+ {
+ for (Selector selector : selectors)
+ selector.addInput(protocolVersion, rs);
+ }
+ };
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
index 0000000,0000000..e6c8979
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
@@@ -1,0 -1,0 +1,118 @@@
++package org.apache.cassandra.cql3.selection;
++
++import java.util.LinkedHashSet;
++import java.util.List;
++
++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.config.ColumnDefinition;
++import org.apache.cassandra.cql3.ColumnSpecification;
++
++/**
++ * Separately maintains the ColumnSpecifications and their mappings to underlying
++ * columns as we may receive null mappings. This occurs where a query result
++ * includes a column specification which does not map to any particular real
++ * column, e.g. COUNT queries or where no-arg functions like now() are used
++ */
++public class SelectionColumnMapping implements SelectionColumns
++{
++ // Uses a LinkedHashSet as both order and uniqueness need to be preserved
++ private final LinkedHashSet<ColumnSpecification> columnSpecifications;
++ private final HashMultimap<ColumnSpecification, ColumnDefinition> columnMappings;
++
++ private SelectionColumnMapping()
++ {
++ this.columnSpecifications = new LinkedHashSet<>();
++ this.columnMappings = HashMultimap.create();
++ }
++
++ protected static SelectionColumnMapping newMapping()
++ {
++ return new SelectionColumnMapping();
++ }
++
++ protected static SelectionColumnMapping simpleMapping(List<ColumnDefinition> columnDefinitions)
++ {
++ SelectionColumnMapping mapping = new SelectionColumnMapping();
++ for (ColumnDefinition def: columnDefinitions)
++ mapping.addMapping(def, def);
++ return mapping;
++ }
++
++ protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, ColumnDefinition column)
++ {
++ columnSpecifications.add(colSpec);
++ // some AbstractFunctionSelector impls do not map directly to an underlying column
++ // so don't record a mapping in that case
++ if (null != column)
++ columnMappings.put(colSpec, column);
++ return this;
++ }
++
++ public List<ColumnSpecification> getColumnSpecifications()
++ {
++ // return a mutable copy as we may add extra columns
++ // for ordering (CASSANDRA-4911 & CASSANDRA-8286)
++ return Lists.newArrayList(columnSpecifications);
++ }
++
++ public Multimap<ColumnSpecification, ColumnDefinition> getMappings()
++ {
++ return Multimaps.unmodifiableMultimap(columnMappings);
++ }
++
++ public boolean equals(Object obj)
++ {
++ if (obj == null)
++ return false;
++
++ if (!(obj instanceof SelectionColumnMapping))
++ return false;
++
++ return Objects.equal(this.columnMappings, ((SelectionColumnMapping)obj).columnMappings);
++ }
++
++ public int hashCode()
++ {
++ return Objects.hashCode(columnMappings);
++ }
++
++ public String toString()
++ {
++ final Function<ColumnDefinition, String> getDefName = new Function<ColumnDefinition, String>()
++ {
++ public String apply(ColumnDefinition def)
++ {
++ return def.name.toString();
++ }
++ };
++ final Function<ColumnSpecification, String> colSpecToMappingString = new Function<ColumnSpecification, String>()
++ {
++ public String apply(ColumnSpecification colSpec)
++ {
++ StringBuilder builder = new StringBuilder();
++ builder.append(colSpec.name.toString());
++ if (columnMappings.containsKey(colSpec))
++ {
++ builder.append(":[");
++ builder.append(Joiner.on(',').join(Iterables.transform(columnMappings.get(colSpec), getDefName)));
++ builder.append("]");
++ }
++ else
++ {
++ builder.append(":[]");
++ }
++ return builder.toString();
++ }
++ };
++
++ StringBuilder builder = new StringBuilder();
++ builder.append("{ ");
++ builder.append(Joiner.on(", ").join(Iterables.transform(columnSpecifications, colSpecToMappingString)));
++ builder.append(" }");
++ return builder.toString();
++ }
++}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/1f8516da/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
index 0000000,0000000..af334e6
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
@@@ -1,0 -1,0 +1,18 @@@
++package org.apache.cassandra.cql3.selection;
++
++import java.util.List;
++
++import com.google.common.collect.Multimap;
++
++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, ColumnDefinition> getMappings();
++}
[05/12] cassandra git commit: Merge branch 'cassandra-2.0' into
cassandra-2.1
Posted by sa...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/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 <sa...@beobal.com>
Authored: Thu Jun 18 17:32:04 2015 +0100
Committer: Sam Tunnicliffe <sa...@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();
- }
- }
+ }
[06/12] cassandra git commit: Merge branch 'cassandra-2.0' into
cassandra-2.1
Posted by sa...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/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.1
Commit: 0452e74f59182aee515cc6920e35287def86c9fe
Parents: 9966419 f32cff8
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 18 17:32:04 2015 +0100
Committer: Sam Tunnicliffe <sa...@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();
- }
- }
+ }