You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sa...@apache.org on 2015/07/02 12:34:34 UTC

[07/10] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

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/7473877e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7473877e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7473877e

Branch: refs/heads/trunk
Commit: 7473877eeaec2772effcfcf855b378bc4ca92789
Parents: b757db1 2a294e4
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jul 2 11:22:18 2015 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jul 2 11:25:28 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/cql3/ResultSet.java    |   2 +-
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../cql3/statements/SelectStatement.java        |   2 +-
 .../cassandra/cql3/statements/Selection.java    |  67 ++++++--
 .../cql3/statements/SelectionColumnMapping.java |  52 +++++-
 .../statements/SelectionColumnMappingTest.java  | 170 ++++++++++++++++---
 7 files changed, 246 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7473877e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 25f7c1d,07de84c..b316aa5
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,13 -1,5 +1,14 @@@
 -2.0.17
 +2.1.8
 + * Ensure memtable book keeping is not corrupted in the event we shrink usage (CASSANDRA-9681)
 + * Update internal python driver for cqlsh (CASSANDRA-9064)
 + * Fix IndexOutOfBoundsException when inserting tuple with too many
 +   elements using the string literal notation (CASSANDRA-9559)
 + * Allow JMX over SSL directly from nodetool (CASSANDRA-9090)
 + * Fix incorrect result for IN queries where column not found (CASSANDRA-9540)
 + * Enable describe on indices (CASSANDRA-7814)
 + * ColumnFamilyStore.selectAndReference may block during compaction (CASSANDRA-9637)
 +Merged from 2.0:
+  * Bug fixes to resultset metadata construction (CASSANDRA-9636)
   * Fix setting 'durable_writes' in ALTER KEYSPACE (CASSANDRA-9560)
   * Avoid ballot clash in Paxos (CASSANDRA-9649)
   * Improve trace messages for RR (CASSANDRA-9479)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7473877e/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7473877e/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 3838909,c731cd4..876c5e4
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -598,7 -672,9 +598,7 @@@ public abstract class ModificationState
          Selection selection;
          if (columnsWithConditions == null)
          {
-             selection = Selection.wildcard(cfm);
 -            selection = Selection.wildcard(cfDef, false, null);
++            selection = Selection.wildcard(cfm, false, null);
          }
          else
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7473877e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 6fea8cb,aaf9579..7241088
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -1506,8 -1537,8 +1506,8 @@@ public class SelectStatement implement
                  throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
  
              Selection selection = selectClause.isEmpty()
-                                 ? Selection.wildcard(cfm)
 -                                ? Selection.wildcard(cfDef, parameters.isCount, parameters.countAlias)
 -                                : Selection.fromSelectors(cfDef, selectClause);
++                                ? Selection.wildcard(cfm, parameters.isCount, parameters.countAlias)
 +                                : Selection.fromSelectors(cfm, selectClause);
  
              SelectStatement stmt = new SelectStatement(cfm, boundNames.size(), parameters, selection, prepareLimit(boundNames));
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7473877e/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/Selection.java
index 83cbfe8,0bad973..d29b917
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@@ -20,7 -20,9 +20,10 @@@ package org.apache.cassandra.cql3.state
  import java.nio.ByteBuffer;
  import java.util.*;
  
 +import com.google.common.collect.Iterators;
+ import com.google.common.collect.Lists;
+ 
+ import org.apache.cassandra.cql3.ColumnSpecification;
  
  import org.apache.cassandra.cql3.*;
  import org.apache.cassandra.cql3.functions.Function;
@@@ -47,11 -44,10 +50,25 @@@ public abstract class Selectio
      private final boolean collectTimestamps;
      private final boolean collectTTLs;
  
-     protected Selection(List<ColumnDefinition> columns, SelectionColumnMapping columnMapping, 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, columnMapping, columnMapping.getColumnSpecifications(), collectTimestamps, collectTTLs);
++    }
++
++    // Alternative ctor for when we need to pass the List<ColumnSpecification> directly,
++    // rather than getting it from columnMapping. This is to handle COUNT correctly.
++    protected Selection(List<ColumnDefinition> columns,
++                        SelectionColumnMapping columnMapping,
++                        List<ColumnSpecification> columnSpecifications,
++                        boolean collectTimestamps,
++                        boolean collectTTLs)
      {
          this.columns = columns;
          this.columnMapping = columnMapping;
-         this.metadata = new ResultSet.Metadata(columnMapping.getColumnSpecifications());
++        this.metadata = new ResultSet.Metadata(columnSpecifications);
          this.collectTimestamps = collectTimestamps;
          this.collectTTLs = collectTTLs;
      }
@@@ -64,29 -60,23 +81,31 @@@
  
      public ResultSet.Metadata getResultMetadata()
      {
 -        return new ResultSet.Metadata(columnMapping.getColumnSpecifications());
 +        return metadata;
      }
  
-     public static Selection wildcard(CFMetaData cfm)
 -    public static Selection wildcard(CFDefinition cfDef, boolean isCount, ColumnIdentifier countAlias)
++    public static Selection wildcard(CFMetaData cfm, boolean isCount, ColumnIdentifier countAlias)
      {
-         List<ColumnDefinition> all = new ArrayList<ColumnDefinition>(cfm.allColumns().size());
-         Iterators.addAll(all, cfm.allColumnsInSelectOrder());
-         return new SimpleSelection(all, true);
 -        SelectionColumnMapping columnMapping = isCount ? SelectionColumnMapping.countMapping(cfDef, countAlias)
 -                                                       : SelectionColumnMapping.simpleMapping(cfDef);
++        List<ColumnDefinition> allColumns = Lists.newArrayList(cfm.allColumnsInSelectOrder());
++        SelectionColumnMapping columnMapping = isCount ? SelectionColumnMapping.countMapping(cfm, countAlias)
++                                                       : SelectionColumnMapping.simpleMapping(allColumns);
+ 
 -        return new SimpleSelection(Lists.newArrayList(cfDef), columnMapping, true, isCount);
++        return new SimpleSelection(allColumns, columnMapping, true, isCount);
      }
  
 -    public static Selection forColumns(List<CFDefinition.Name> columns)
 +    public static Selection forColumns(List<ColumnDefinition> columns)
      {
-         return new SimpleSelection(columns, false);
+         return new SimpleSelection(columns);
      }
  
 -    private static boolean selectionsNeedProcessing(List<RawSelector> rawSelectors)
 +    public int addColumnForOrdering(ColumnDefinition c)
 +    {
 +        columns.add(c);
 +        metadata.addNonSerializedColumn(c);
 +        return columns.size() - 1;
 +    }
 +
 +    private static boolean requiresProcessing(List<RawSelector> rawSelectors)
      {
          for (RawSelector rawSelector : rawSelectors)
          {
@@@ -117,13 -114,12 +136,13 @@@
      {
          if (selectable instanceof ColumnIdentifier)
          {
-             ColumnDefinition def = cfm.getColumnDefinition((ColumnIdentifier)selectable);
 -            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 (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)
          {
@@@ -245,18 -209,17 +264,18 @@@
              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);
 +                columnMapping.addMapping(rawSelector.alias == null ? def : makeAliasSpec(cfm,
 +                                                                                         def.type,
 +                                                                                         rawSelector.alias),
 +                                         def);
              }
-             return new SimpleSelection(defs, columnMapping, false);
 -            return new SimpleSelection(names, columnMapping);
++            return new SimpleSelection(defs, columnMapping);
          }
      }
  
@@@ -283,9 -260,14 +302,14 @@@
          return new ResultSetBuilder(now);
      }
  
+     protected List<ColumnSpecification> getColumnSpecifications()
+     {
+         return columnMapping.getColumnSpecifications();
+     }
+ 
 -    private static ByteBuffer value(Column c)
 +    private static ByteBuffer value(Cell c)
      {
 -        return (c instanceof CounterColumn)
 +        return (c instanceof CounterCell)
              ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
              : c.value();
      }
@@@ -363,21 -345,28 +387,36 @@@
      private static class SimpleSelection extends Selection
      {
          private final boolean isWildcard;
+         private final boolean isCount;
+ 
 -        public SimpleSelection(List<CFDefinition.Name> columns)
++        public SimpleSelection(List<ColumnDefinition> columns)
+         {
+             this(columns, SelectionColumnMapping.simpleMapping(columns), false, false);
+         }
  
-         public SimpleSelection(List<ColumnDefinition> columns, boolean isWildcard)
 -        public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping)
++        public SimpleSelection(List<ColumnDefinition> columns, SelectionColumnMapping columnMapping)
          {
-             this(columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
+             this(columns, columnMapping, false, false);
          }
  
-         public SimpleSelection(List<ColumnDefinition> columns, SelectionColumnMapping columnMapping, boolean isWildcard)
 -        public SimpleSelection(List<CFDefinition.Name> columns, SelectionColumnMapping columnMapping, boolean wildcard, boolean isCount)
++        public SimpleSelection(List<ColumnDefinition> columns,
++                               SelectionColumnMapping columnMapping,
++                               boolean wildcard,
++                               boolean isCount)
          {
              /*
               * 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, columnMapping, false, false);
-             this.isWildcard = isWildcard;
++            // the List<ColumnSpecification> we pass depends on whether this is a COUNT or not
++            super(columns,
++                  columnMapping,
++                  isCount ? new ArrayList<ColumnSpecification>(columns) : columnMapping.getColumnSpecifications(),
++                  false,
++                  false);
+             this.isWildcard = wildcard;
+             this.isCount = isCount;
          }
  
          protected List<ByteBuffer> handleRow(ResultSetBuilder rs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7473877e/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
index 4a6955f,260f052..aba2631
--- a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
@@@ -4,16 -4,20 +4,20 @@@ 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.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
+ import org.apache.cassandra.cql3.ColumnIdentifier;
 -
 +import org.apache.cassandra.cql3.ColumnSpecification;
+ import org.apache.cassandra.cql3.ResultSet;
+ import org.apache.cassandra.db.marshal.LongType;
  
 -import org.apache.cassandra.cql3.CFDefinition;
 -import org.apache.cassandra.cql3.ColumnSpecification;
 -
  public class SelectionColumnMapping implements SelectionColumns
  {
-     // Uses LinkedHashMultimap because ordering of keys must be maintained
-     private final LinkedHashMultimap<ColumnSpecification, ColumnDefinition> columnMappings;
+     private final ArrayList<ColumnSpecification> columnSpecifications;
 -    private final HashMultimap<ColumnSpecification, CFDefinition.Name> columnMappings;
++    private final HashMultimap<ColumnSpecification, ColumnDefinition> columnMappings;
  
      private SelectionColumnMapping()
      {
@@@ -25,22 -30,38 +30,38 @@@
          return new SelectionColumnMapping();
      }
  
-     protected static SelectionColumnMapping simpleMapping(List<ColumnDefinition> columnDefinitions)
 -    protected static SelectionColumnMapping countMapping(CFDefinition cfDef, ColumnIdentifier countAlias)
++    protected static SelectionColumnMapping countMapping(CFMetaData cfm, ColumnIdentifier countAlias)
+     {
 -        ColumnSpecification spec = new ColumnSpecification(cfDef.cfm.ksName,
 -                                                           cfDef.cfm.cfName,
++        ColumnSpecification spec = new ColumnSpecification(cfm.ksName,
++                                                           cfm.cfName,
+                                                            countAlias == null ? ResultSet.COUNT_COLUMN
+                                                                               : countAlias,
+                                                            LongType.instance);
+ 
 -        return new SelectionColumnMapping().addMapping(spec, Collections.<CFDefinition.Name>emptyList());
++        return new SelectionColumnMapping().addMapping(spec, Collections.<ColumnDefinition>emptyList());
+     }
+ 
 -    protected static SelectionColumnMapping simpleMapping(Iterable<CFDefinition.Name> columnDefinitions)
++    protected static SelectionColumnMapping simpleMapping(Iterable<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);
+         columnSpecifications.add(colSpec);
+         // functions without arguments do not map to any column, so don't
+         // record any mapping in that case
+         if (column != null)
+             columnMappings.put(colSpec, column);
          return this;
      }
  
 -    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<CFDefinition.Name> columns)
 +    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<ColumnDefinition> columns)
      {
+         columnSpecifications.add(colSpec);
          columnMappings.putAll(colSpec, columns);
          return this;
      }
@@@ -49,10 -70,10 +70,10 @@@
      {
          // return a mutable copy as we may add extra columns
          // for ordering (CASSANDRA-4911 & CASSANDRA-8286)
-         return Lists.newArrayList(columnMappings.keySet());
+         return Lists.newArrayList(columnSpecifications);
      }
  
 -    public Multimap<ColumnSpecification, CFDefinition.Name> getMappings()
 +    public Multimap<ColumnSpecification, ColumnDefinition> getMappings()
      {
          return Multimaps.unmodifiableMultimap(columnMappings);
      }
@@@ -62,10 -83,12 +83,12 @@@
          if (obj == null)
              return false;
  
 -        if (!(obj instanceof SelectionColumns))
 +        if (!(obj instanceof SelectionColumnMapping))
              return false;
  
-         return Objects.equal(this.columnMappings, ((SelectionColumnMapping) obj).columnMappings);
+         SelectionColumns other = (SelectionColumns)obj;
 -        return Objects.equals(columnMappings, other.getMappings())
 -            && Objects.equals(columnSpecifications, other.getColumnSpecifications());
++        return Objects.equal(columnMappings, other.getMappings())
++            && Objects.equal(columnSpecifications, other.getColumnSpecifications());
      }
  
      public int hashCode()
@@@ -100,8 -135,7 +134,8 @@@
          builder.append(Joiner.on(", ")
                               .join(Iterables.transform(columnMappings.asMap().entrySet(),
                                                         mappingEntryToString)));
-         builder.append(" }");
+         builder.append("] }");
          return builder.toString();
      }
 +
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7473877e/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
index ccd3570,3acb66a..f01a94d
--- a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
@@@ -6,27 -6,34 +6,31 @@@ 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.DatabaseDescriptor;
  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.dht.ByteOrderedPartitioner;
+ import org.apache.cassandra.exceptions.RequestExecutionException;
  import org.apache.cassandra.exceptions.RequestValidationException;
  import org.apache.cassandra.service.ClientState;
+ import org.apache.cassandra.service.QueryState;
  
 -import static org.apache.cassandra.cql3.QueryProcessor.process;
  import static org.junit.Assert.assertEquals;
  import static org.junit.Assert.assertTrue;
  
 -public class SelectionColumnMappingTest
 +public class SelectionColumnMappingTest extends CQLTester
  {
 -    private static final CFDefinition.Name NULL_DEF = null;
 -    static String KEYSPACE = "selection_column_mapping_test_ks";
 -    String tableName = "test_table";
++    private static final ColumnDefinition NULL_DEF = null;
++
 +    String tableName;
 +    String typeName;
  
      @BeforeClass
 -    public static void setupSchema() throws Throwable
 +    public static void setUpClass()
      {
 -        SchemaLoader.loadSchema();
 -        executeSchemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s " +
 -                                          "WITH replication = {'class': 'SimpleStrategy', " +
 -                                          "                    'replication_factor': '1'}",
 -                                          KEYSPACE));
 +        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
      }
  
      @Test
@@@ -34,13 -41,13 +38,14 @@@
      {
          // 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)");
 -        insert("INSERT INTO %s (k, v1 ,v2) VALUES (1, 1, 'foo')");
 +        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 + ">)");
++        execute("INSERT INTO %s (k, v1 ,v2, v3) VALUES (1, 1, 'foo', {f1:1, f2:'bar'})");
+ 
          testSimpleTypes();
          testWildcard();
          testSimpleTypesWithAliases();
@@@ -58,44 -71,46 +71,47 @@@
      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"));
+         // we don't use verify like with the other tests because this query will produce no results
+         SelectStatement statement = getSelect("SELECT token(a,b) FROM %s");
+         verifyColumnMapping(expected, statement);
+         statement.executeInternal(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
      }
  
      private void testSimpleTypes() throws Throwable
      {
          // 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"));
+         verify(expected, "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"));
+         verify(expected, "SELECT * FROM %s");
      }
  
      private void testSimpleTypesWithAliases() throws Throwable
@@@ -105,51 -120,25 +121,51 @@@
          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"));
+         verify(expected, "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"));
++        verify(expected, "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"));
++        verify(expected, "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"));
+         verify(expected, "SELECT writetime(v1), ttl(v2) FROM %s");
      }
  
      private void testWritetimeAndTTLWithAliases() throws Throwable
@@@ -158,11 -147,11 +174,11 @@@
          // 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"));
+         verify(expected, "SELECT writetime(v1) AS wt_alias, ttl(v2) AS ttl_alias FROM %s");
      }
  
      private void testFunction() throws Throwable
@@@ -171,10 -160,10 +187,10 @@@
          // 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"));
+         verify(expected, "SELECT intasblob(v1) FROM %s");
      }
  
      private void testFunctionWithAlias() throws Throwable
@@@ -183,10 -172,84 +199,84 @@@
          // 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"));
+         verify(expected, "SELECT intasblob(v1) AS fn_alias FROM %s");
+     }
+ 
+     public void testNoArgumentFunction() throws Throwable
+     {
+         SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                           .addMapping(columnSpecification("now()",
+                                                                                           TimeUUIDType.instance),
+                                                                       NULL_DEF);
+         verify(expected, "SELECT now() FROM %s");
+     }
+ 
+     public void testNestedFunctionsWithArguments() throws Throwable
+     {
+         SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                           .addMapping(columnSpecification("blobasint(intasblob(v1))",
+                                                                                           Int32Type.instance),
+                                                                       columnDefinition("v1"));
+         verify(expected, "SELECT blobasint(intasblob(v1)) FROM %s");
+     }
+ 
+     public void testNestedFunctions() throws Throwable
+     {
+         SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                           .addMapping(columnSpecification("unixtimestampof(now())",
+                                                                                           LongType.instance),
+                                                                       NULL_DEF);
+         verify(expected, "SELECT unixtimestampof(now()) FROM %s");
+     }
+ 
+     public void testCount() throws Throwable
+     {
+         SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                           .addMapping(columnSpecification("count", LongType.instance),
+                                                                       NULL_DEF);
+         verify(expected, "SELECT count(*) FROM %s");
+         verify(expected, "SELECT count(1) FROM %s");
+ 
+         expected = SelectionColumnMapping.newMapping()
+                                          .addMapping(columnSpecification("other_count", LongType.instance), NULL_DEF);
+         verify(expected, "SELECT count(*) AS other_count FROM %s");
+         verify(expected, "SELECT count(1) AS other_count FROM %s");
+     }
+ 
+     public void testDuplicateFunctionsWithoutAliases() throws Throwable
+     {
+         // where duplicate functions are present, the ColumnSpecification list will
+         // contain an entry per-duplicate but the mappings will be deduplicated (i.e.
+         // a single mapping k/v pair regardless of the number of duplicates)
+         ColumnSpecification spec = columnSpecification("intasblob(v1)", BytesType.instance);
+         SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                           .addMapping(spec, columnDefinition("v1"))
+                                                           .addMapping(spec, columnDefinition("v1"));
+         verify(expected, "SELECT intasblob(v1), intasblob(v1) FROM %s");
+     }
+ 
+     public void testDuplicateFunctionsWithAliases() throws Throwable
+     {
+         // where duplicate functions are present with distinct aliases, they are
+         // represented as any other set of distinct columns would be - an entry
+         // in theColumnSpecification list and a separate k/v mapping for each
+         SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                           .addMapping(columnSpecification("blob_1", BytesType.instance),
+                                                                       columnDefinition("v1"))
+                                                           .addMapping(columnSpecification("blob_2", BytesType.instance),
+                                                                       columnDefinition("v1"));
+         verify(expected, "SELECT intasblob(v1) AS blob_1, intasblob(v1) AS blob_2 FROM %s");
+     }
+ 
+     public void testSelectDistinct() throws Throwable
+     {
+         SelectionColumns expected = SelectionColumnMapping.newMapping().addMapping(columnDefinition("k"),
+                                                                                    columnDefinition("k"));
+         verify(expected, "SELECT DISTINCT k FROM %s");
+ 
      }
  
      private void testMultipleAliasesOnSameColumn() throws Throwable
@@@ -195,11 -258,22 +285,23 @@@
          // 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"));
+         verify(expected, "SELECT v1 AS alias_1, v1 AS alias_2 FROM %s");
+     }
+ 
+     private void testMultipleUnaliasedSelectionOfSameColumn() throws Throwable
+     {
+         // simple column identifiers without aliases are represented in
+         // ResultSet.Metadata by the underlying ColumnDefinition
 -        CFDefinition.Name v1 = columnDefinition("v1");
++        ColumnDefinition v1 = columnDefinition("v1");
+         SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                           .addMapping(v1, v1)
+                                                           .addMapping(v1, v1);
+ 
+         verify(expected, "SELECT v1, v1 FROM %s");
      }
  
      private void testMixedColumnTypes() throws Throwable
@@@ -207,39 -281,52 +309,59 @@@
          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);
  
 -        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"))
 +                                                                .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," +
-                                                               "       v3.f1," +
-                                                               "       v3.f2 AS f2_alias," +
-                                                               "       v3" +
-                                                               " FROM %s"));
++                                                                .addMapping(columnDefinition("v3"), columnDefinition("v3"));
+ 
+         verify(expected, "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
+     private void verify(SelectionColumns expected, String query) throws Throwable
+     {
+         SelectStatement statement = getSelect(query);
+         verifyColumnMapping(expected, statement);
+         checkExecution(statement, expected.getColumnSpecifications());
+     }
+ 
+     private void checkExecution(SelectStatement statement, List<ColumnSpecification> expectedResultColumns)
+     throws RequestExecutionException, RequestValidationException
+     {
 -        UntypedResultSet rs = new UntypedResultSet(statement.executeInternal(QueryState.forInternalCalls(),
 -                                                                             QueryOptions.DEFAULT).result);
++        UntypedResultSet rs = UntypedResultSet.create(statement.executeInternal(QueryState.forInternalCalls(),
++                                                                                QueryOptions.DEFAULT).result);
+ 
+         assertEquals(expectedResultColumns, rs.one().getColumns());
+     }
+ 
+     private SelectStatement getSelect(String query) throws RequestValidationException
      {
          CQLStatement statement = QueryProcessor.getStatement(String.format(query, KEYSPACE + "." + tableName),
                                                               ClientState.forInternalCalls()).statement;
          assertTrue(statement instanceof SelectStatement);
-         return ((SelectStatement)statement).getSelection().getColumnMapping();
+         return (SelectStatement)statement;
+     }
+ 
+     private void verifyColumnMapping(SelectionColumns expected, SelectStatement select)
+     {
+         assertEquals(expected, select.getSelection().getColumnMapping());
      }
  
 -    private CFDefinition.Name columnDefinition(String name)
 +    private ColumnDefinition columnDefinition(String name)
      {
          return Schema.instance.getCFMetaData(KEYSPACE, tableName)
 -                              .getCfDef()
 -                              .get(new ColumnIdentifier(name, true));
 +                              .getColumnDefinition(new ColumnIdentifier(name, true));
  
      }