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 &lt;selector&gt; 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 &lt;selector&gt; 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();
 -        }
 -    }
+ }