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/07/02 12:34:31 UTC
[04/10] cassandra git commit: Bug fixes to SelectionColumnMapping
Bug fixes to SelectionColumnMapping
Patch and review by Benjamin Lerer and Sam Tunnicliffe for
CASSANDRA-9636
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2a294e45
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2a294e45
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2a294e45
Branch: refs/heads/trunk
Commit: 2a294e45aa023af28ccc179c5f41410940ef40d7
Parents: ccec307
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jul 2 11:18:21 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jul 2 11:18:21 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../org/apache/cassandra/cql3/ResultSet.java | 2 +-
.../cql3/statements/ModificationStatement.java | 2 +-
.../cql3/statements/SelectStatement.java | 2 +-
.../cassandra/cql3/statements/Selection.java | 51 ++++--
.../cql3/statements/SelectionColumnMapping.java | 55 +++++--
.../statements/SelectionColumnMappingTest.java | 158 +++++++++++++++++--
7 files changed, 229 insertions(+), 42 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a294e45/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index beebaf3..07de84c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.0.17
+ * Bug fixes to resultset metadata construction (CASSANDRA-9636)
* Fix setting 'durable_writes' in ALTER KEYSPACE (CASSANDRA-9560)
* Avoid ballot clash in Paxos (CASSANDRA-9649)
* Improve trace messages for RR (CASSANDRA-9479)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a294e45/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 659ed50..74a276b 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -37,7 +37,7 @@ import org.apache.cassandra.service.pager.PagingState;
public class ResultSet
{
public static final Codec codec = new Codec();
- private static final ColumnIdentifier COUNT_COLUMN = new ColumnIdentifier("count", false);
+ public static final ColumnIdentifier COUNT_COLUMN = new ColumnIdentifier("count", false);
public final Metadata metadata;
public final List<List<ByteBuffer>> rows;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a294e45/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 3852920..c731cd4 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -674,7 +674,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
Selection selection;
if (columnsWithConditions == null)
{
- selection = Selection.wildcard(cfDef);
+ selection = Selection.wildcard(cfDef, false, null);
}
else
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a294e45/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 341ce81..aaf9579 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -1537,7 +1537,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
Selection selection = selectClause.isEmpty()
- ? Selection.wildcard(cfDef)
+ ? Selection.wildcard(cfDef, parameters.isCount, parameters.countAlias)
: Selection.fromSelectors(cfDef, selectClause);
SelectStatement stmt = new SelectStatement(cfm, boundNames.size(), parameters, selection, prepareLimit(boundNames));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a294e45/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 50a34bf..0bad973 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -20,7 +20,9 @@ package org.apache.cassandra.cql3.statements;
import java.nio.ByteBuffer;
import java.util.*;
-import com.google.common.collect.*;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
import org.apache.cassandra.cql3.*;
import org.apache.cassandra.cql3.functions.Function;
@@ -61,17 +63,17 @@ public abstract class Selection
return new ResultSet.Metadata(columnMapping.getColumnSpecifications());
}
- public static Selection wildcard(CFDefinition cfDef)
+ public static Selection wildcard(CFDefinition cfDef, boolean isCount, ColumnIdentifier countAlias)
{
- List<CFDefinition.Name> all = new ArrayList<CFDefinition.Name>();
- for (CFDefinition.Name name : cfDef)
- all.add(name);
- return new SimpleSelection(all, true);
+ SelectionColumnMapping columnMapping = isCount ? SelectionColumnMapping.countMapping(cfDef, countAlias)
+ : SelectionColumnMapping.simpleMapping(cfDef);
+
+ return new SimpleSelection(Lists.newArrayList(cfDef), columnMapping, true, isCount);
}
public static Selection forColumns(List<CFDefinition.Name> columns)
{
- return new SimpleSelection(columns, false);
+ return new SimpleSelection(columns);
}
private static boolean selectionsNeedProcessing(List<RawSelector> rawSelectors)
@@ -217,7 +219,7 @@ public abstract class Selection
rawSelector.alias),
name);
}
- return new SimpleSelection(names, columnMapping, false);
+ return new SimpleSelection(names, columnMapping);
}
}
@@ -258,6 +260,11 @@ public abstract class Selection
return new ResultSetBuilder(now);
}
+ protected List<ColumnSpecification> getColumnSpecifications()
+ {
+ return columnMapping.getColumnSpecifications();
+ }
+
private static ByteBuffer value(Column c)
{
return (c instanceof CounterColumn)
@@ -284,7 +291,7 @@ public abstract class Selection
private ResultSetBuilder(long now)
{
- this.resultSet = new ResultSet(columnMapping.getColumnSpecifications());
+ this.resultSet = new ResultSet(getColumnSpecifications());
this.timestamps = collectTimestamps ? new long[columns.size()] : null;
this.ttls = collectTTLs ? new int[columns.size()] : null;
this.now = now;
@@ -338,13 +345,19 @@ public abstract class Selection
private static class SimpleSelection extends Selection
{
private final boolean isWildcard;
+ private final boolean isCount;
+
+ public SimpleSelection(List<CFDefinition.Name> columns)
+ {
+ this(columns, SelectionColumnMapping.simpleMapping(columns), false, false);
+ }
- public SimpleSelection(List<CFDefinition.Name> columns, boolean isWildcard)
+ public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping)
{
- this(columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
+ this(columns, columnMapping, false, false);
}
- public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping, boolean isWildcard)
+ public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping, boolean wildcard, boolean isCount)
{
/*
* In theory, even a simple selection could have multiple time the same column, so we
@@ -352,7 +365,8 @@ public abstract class Selection
* get much duplicate in practice, it's more efficient not to bother.
*/
super(columns, columnMapping, false, false);
- this.isWildcard = isWildcard;
+ this.isWildcard = wildcard;
+ this.isCount = isCount;
}
protected List<ByteBuffer> handleRow(ResultSetBuilder rs)
@@ -360,6 +374,17 @@ public abstract class Selection
return rs.current;
}
+ /**
+ * This method is overridden to make sure that the ResultSet is build properly in the case of a count
+ * query.
+ */
+ @Override
+ protected List<ColumnSpecification> getColumnSpecifications()
+ {
+ return isCount ? new ArrayList<ColumnSpecification>(this.getColumns())
+ : super.getColumnSpecifications();
+ }
+
@Override
public boolean isWildcard()
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a294e45/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
index d09612f..260f052 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
@@ -6,18 +6,23 @@ import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.collect.*;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.db.marshal.LongType;
+
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 final ArrayList<ColumnSpecification> columnSpecifications;
+ private final HashMultimap<ColumnSpecification, CFDefinition.Name> columnMappings;
private SelectionColumnMapping()
{
- this.columnMappings = LinkedHashMultimap.create();
+ this.columnSpecifications = new ArrayList<>();
+ this.columnMappings = HashMultimap.create();
}
protected static SelectionColumnMapping newMapping()
@@ -25,7 +30,18 @@ public class SelectionColumnMapping implements SelectionColumns
return new SelectionColumnMapping();
}
- protected static SelectionColumnMapping simpleMapping(List<CFDefinition.Name> columnDefinitions)
+ protected static SelectionColumnMapping countMapping(CFDefinition cfDef, ColumnIdentifier countAlias)
+ {
+ ColumnSpecification spec = new ColumnSpecification(cfDef.cfm.ksName,
+ cfDef.cfm.cfName,
+ countAlias == null ? ResultSet.COUNT_COLUMN
+ : countAlias,
+ LongType.instance);
+
+ return new SelectionColumnMapping().addMapping(spec, Collections.<CFDefinition.Name>emptyList());
+ }
+
+ protected static SelectionColumnMapping simpleMapping(Iterable<CFDefinition.Name> columnDefinitions)
{
SelectionColumnMapping mapping = new SelectionColumnMapping();
for (CFDefinition.Name def: columnDefinitions)
@@ -35,12 +51,17 @@ public class SelectionColumnMapping implements SelectionColumns
protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, CFDefinition.Name column)
{
- columnMappings.put(colSpec, column);
+ columnSpecifications.add(colSpec);
+ // functions without arguments do not map to any column, so don't
+ // record any mapping in that case
+ if (column != null)
+ columnMappings.put(colSpec, column);
return this;
}
protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<CFDefinition.Name> columns)
{
+ columnSpecifications.add(colSpec);
columnMappings.putAll(colSpec, columns);
return this;
}
@@ -49,7 +70,7 @@ public class SelectionColumnMapping implements SelectionColumns
{
// return a mutable copy as we may add extra columns
// for ordering (CASSANDRA-4911 & CASSANDRA-8286)
- return new ArrayList(columnMappings.keySet());
+ return Lists.newArrayList(columnSpecifications);
}
public Multimap<ColumnSpecification, CFDefinition.Name> getMappings()
@@ -65,7 +86,9 @@ public class SelectionColumnMapping implements SelectionColumns
if (!(obj instanceof SelectionColumns))
return false;
- return Objects.equals(columnMappings, ((SelectionColumns) obj).getMappings());
+ SelectionColumns other = (SelectionColumns)obj;
+ return Objects.equals(columnMappings, other.getMappings())
+ && Objects.equals(columnSpecifications, other.getColumnSpecifications());
}
public int hashCode()
@@ -82,6 +105,7 @@ public class SelectionColumnMapping implements SelectionColumns
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)
@@ -95,12 +119,23 @@ public class SelectionColumnMapping implements SelectionColumns
}
};
+ Function<ColumnSpecification, String> colSpecToString = new Function<ColumnSpecification, String>()
+ {
+ public String apply(ColumnSpecification columnSpecification)
+ {
+ return columnSpecification.name.toString();
+ }
+ };
+
StringBuilder builder = new StringBuilder();
- builder.append("{ ");
+ builder.append("{ Columns:[");
+ builder.append(Joiner.on(",")
+ .join(Iterables.transform(columnSpecifications, colSpecToString )));
+ builder.append("], Mappings:[");
builder.append(Joiner.on(", ")
.join(Iterables.transform(columnMappings.asMap().entrySet(),
mappingEntryToString)));
- builder.append(" }");
+ builder.append("] }");
return builder.toString();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a294e45/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
index 9c31653..3acb66a 100644
--- a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
@@ -11,8 +11,10 @@ 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.RequestExecutionException;
import org.apache.cassandra.exceptions.RequestValidationException;
import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
import static org.apache.cassandra.cql3.QueryProcessor.process;
import static org.junit.Assert.assertEquals;
@@ -20,6 +22,7 @@ import static org.junit.Assert.assertTrue;
public class SelectionColumnMappingTest
{
+ private static final CFDefinition.Name NULL_DEF = null;
static String KEYSPACE = "selection_column_mapping_test_ks";
String tableName = "test_table";
@@ -43,6 +46,8 @@ public class SelectionColumnMappingTest
" k int PRIMARY KEY," +
" v1 int," +
" v2 ascii)");
+ insert("INSERT INTO %s (k, v1 ,v2) VALUES (1, 1, 'foo')");
+
testSimpleTypes();
testWildcard();
testSimpleTypesWithAliases();
@@ -50,8 +55,16 @@ public class SelectionColumnMappingTest
testWritetimeAndTTLWithAliases();
testFunction();
testFunctionWithAlias();
+ testNoArgumentFunction();
+ testNestedFunctions();
+ testNestedFunctionsWithArguments();
+ testCount();
+ testDuplicateFunctionsWithoutAliases();
+ testDuplicateFunctionsWithAliases();
+ testSelectDistinct();
testMultipleAliasesOnSameColumn();
testMixedColumnTypes();
+ testMultipleUnaliasedSelectionOfSameColumn();
}
@Test
@@ -64,7 +77,10 @@ public class SelectionColumnMappingTest
SelectionColumns expected = SelectionColumnMapping.newMapping()
.addMapping(tokenSpec, columnDefinitions("a", "b"));
- assertEquals(expected, extractColumnMappingFromSelect("SELECT token(a,b) FROM %s"));
+ // we don't use verify like with the other tests because this query will produce no results
+ SelectStatement statement = getSelect("SELECT token(a,b) FROM %s");
+ verifyColumnMapping(expected, statement);
+ statement.executeInternal(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
}
private void testSimpleTypes() throws Throwable
@@ -79,7 +95,7 @@ public class SelectionColumnMappingTest
.addMapping(v1Def, columnDefinition("v1"))
.addMapping(v2Def, columnDefinition("v2"));
- assertEquals(expected, extractColumnMappingFromSelect("SELECT k, v1, v2 FROM %s"));
+ verify(expected, "SELECT k, v1, v2 FROM %s");
}
private void testWildcard() throws Throwable
@@ -94,7 +110,7 @@ public class SelectionColumnMappingTest
.addMapping(v1Def, columnDefinition("v1"))
.addMapping(v2Def, columnDefinition("v2"));
- assertEquals(expected, extractColumnMappingFromSelect("SELECT * FROM %s"));
+ verify(expected, "SELECT * FROM %s");
}
private void testSimpleTypesWithAliases() throws Throwable
@@ -109,8 +125,7 @@ public class SelectionColumnMappingTest
.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"));
+ verify(expected, "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_alias FROM %s");
}
private void testWritetimeAndTTL() throws Throwable
@@ -123,7 +138,7 @@ public class SelectionColumnMappingTest
.addMapping(wtSpec, columnDefinition("v1"))
.addMapping(ttlSpec, columnDefinition("v2"));
- assertEquals(expected, extractColumnMappingFromSelect("SELECT writetime(v1), ttl(v2) FROM %s"));
+ verify(expected, "SELECT writetime(v1), ttl(v2) FROM %s");
}
private void testWritetimeAndTTLWithAliases() throws Throwable
@@ -136,7 +151,7 @@ public class SelectionColumnMappingTest
.addMapping(wtSpec, columnDefinition("v1"))
.addMapping(ttlSpec, columnDefinition("v2"));
- assertEquals(expected, extractColumnMappingFromSelect("SELECT writetime(v1) AS wt_alias, ttl(v2) AS ttl_alias FROM %s"));
+ verify(expected, "SELECT writetime(v1) AS wt_alias, ttl(v2) AS ttl_alias FROM %s");
}
private void testFunction() throws Throwable
@@ -148,7 +163,7 @@ public class SelectionColumnMappingTest
SelectionColumns expected = SelectionColumnMapping.newMapping()
.addMapping(fnSpec, columnDefinition("v1"));
- assertEquals(expected, extractColumnMappingFromSelect("SELECT intasblob(v1) FROM %s"));
+ verify(expected, "SELECT intasblob(v1) FROM %s");
}
private void testFunctionWithAlias() throws Throwable
@@ -160,7 +175,81 @@ public class SelectionColumnMappingTest
SelectionColumns expected = SelectionColumnMapping.newMapping()
.addMapping(fnSpec, columnDefinition("v1"));
- assertEquals(expected, extractColumnMappingFromSelect("SELECT intasblob(v1) AS fn_alias FROM %s"));
+ verify(expected, "SELECT intasblob(v1) AS fn_alias FROM %s");
+ }
+
+ public void testNoArgumentFunction() throws Throwable
+ {
+ SelectionColumns expected = SelectionColumnMapping.newMapping()
+ .addMapping(columnSpecification("now()",
+ TimeUUIDType.instance),
+ NULL_DEF);
+ verify(expected, "SELECT now() FROM %s");
+ }
+
+ public void testNestedFunctionsWithArguments() throws Throwable
+ {
+ SelectionColumns expected = SelectionColumnMapping.newMapping()
+ .addMapping(columnSpecification("blobasint(intasblob(v1))",
+ Int32Type.instance),
+ columnDefinition("v1"));
+ verify(expected, "SELECT blobasint(intasblob(v1)) FROM %s");
+ }
+
+ public void testNestedFunctions() throws Throwable
+ {
+ SelectionColumns expected = SelectionColumnMapping.newMapping()
+ .addMapping(columnSpecification("unixtimestampof(now())",
+ LongType.instance),
+ NULL_DEF);
+ verify(expected, "SELECT unixtimestampof(now()) FROM %s");
+ }
+
+ public void testCount() throws Throwable
+ {
+ SelectionColumns expected = SelectionColumnMapping.newMapping()
+ .addMapping(columnSpecification("count", LongType.instance),
+ NULL_DEF);
+ verify(expected, "SELECT count(*) FROM %s");
+ verify(expected, "SELECT count(1) FROM %s");
+
+ expected = SelectionColumnMapping.newMapping()
+ .addMapping(columnSpecification("other_count", LongType.instance), NULL_DEF);
+ verify(expected, "SELECT count(*) AS other_count FROM %s");
+ verify(expected, "SELECT count(1) AS other_count FROM %s");
+ }
+
+ public void testDuplicateFunctionsWithoutAliases() throws Throwable
+ {
+ // where duplicate functions are present, the ColumnSpecification list will
+ // contain an entry per-duplicate but the mappings will be deduplicated (i.e.
+ // a single mapping k/v pair regardless of the number of duplicates)
+ ColumnSpecification spec = columnSpecification("intasblob(v1)", BytesType.instance);
+ SelectionColumns expected = SelectionColumnMapping.newMapping()
+ .addMapping(spec, columnDefinition("v1"))
+ .addMapping(spec, columnDefinition("v1"));
+ verify(expected, "SELECT intasblob(v1), intasblob(v1) FROM %s");
+ }
+
+ public void testDuplicateFunctionsWithAliases() throws Throwable
+ {
+ // where duplicate functions are present with distinct aliases, they are
+ // represented as any other set of distinct columns would be - an entry
+ // in theColumnSpecification list and a separate k/v mapping for each
+ SelectionColumns expected = SelectionColumnMapping.newMapping()
+ .addMapping(columnSpecification("blob_1", BytesType.instance),
+ columnDefinition("v1"))
+ .addMapping(columnSpecification("blob_2", BytesType.instance),
+ columnDefinition("v1"));
+ verify(expected, "SELECT intasblob(v1) AS blob_1, intasblob(v1) AS blob_2 FROM %s");
+ }
+
+ public void testSelectDistinct() throws Throwable
+ {
+ SelectionColumns expected = SelectionColumnMapping.newMapping().addMapping(columnDefinition("k"),
+ columnDefinition("k"));
+ verify(expected, "SELECT DISTINCT k FROM %s");
+
}
private void testMultipleAliasesOnSameColumn() throws Throwable
@@ -172,8 +261,19 @@ public class SelectionColumnMappingTest
SelectionColumns expected = SelectionColumnMapping.newMapping()
.addMapping(alias1, columnDefinition("v1"))
.addMapping(alias2, columnDefinition("v1"));
+ verify(expected, "SELECT v1 AS alias_1, v1 AS alias_2 FROM %s");
+ }
- assertEquals(expected, extractColumnMappingFromSelect("SELECT v1 AS alias_1, v1 AS alias_2 FROM %s"));
+ private void testMultipleUnaliasedSelectionOfSameColumn() throws Throwable
+ {
+ // simple column identifiers without aliases are represented in
+ // ResultSet.Metadata by the underlying ColumnDefinition
+ CFDefinition.Name v1 = columnDefinition("v1");
+ SelectionColumns expected = SelectionColumnMapping.newMapping()
+ .addMapping(v1, v1)
+ .addMapping(v1, v1);
+
+ verify(expected, "SELECT v1, v1 FROM %s");
}
private void testMixedColumnTypes() throws Throwable
@@ -187,18 +287,39 @@ public class SelectionColumnMappingTest
.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"));
+ verify(expected, "SELECT k AS k_alias," +
+ " writetime(v1)," +
+ " ttl(v2) as ttl_alias" +
+ " FROM %s");
+ }
+
+ private void verify(SelectionColumns expected, String query) throws Throwable
+ {
+ SelectStatement statement = getSelect(query);
+ verifyColumnMapping(expected, statement);
+ checkExecution(statement, expected.getColumnSpecifications());
+ }
+
+ private void checkExecution(SelectStatement statement, List<ColumnSpecification> expectedResultColumns)
+ throws RequestExecutionException, RequestValidationException
+ {
+ UntypedResultSet rs = new UntypedResultSet(statement.executeInternal(QueryState.forInternalCalls(),
+ QueryOptions.DEFAULT).result);
+
+ assertEquals(expectedResultColumns, rs.one().getColumns());
}
- private SelectionColumns extractColumnMappingFromSelect(String query) throws RequestValidationException
+ private SelectStatement getSelect(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();
+ return (SelectStatement)statement;
+ }
+
+ private void verifyColumnMapping(SelectionColumns expected, SelectStatement select)
+ {
+ assertEquals(expected, select.getSelection().getColumnMapping());
}
private CFDefinition.Name columnDefinition(String name)
@@ -225,6 +346,11 @@ public class SelectionColumnMappingTest
type);
}
+ private void insert(String cql)
+ {
+ QueryProcessor.processInternal(String.format(cql, KEYSPACE + "." + tableName));
+ }
+
private void createTable(String query) throws Throwable
{
executeSchemaChange(String.format(query, KEYSPACE + "." + tableName));