You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2015/11/16 15:09:22 UTC

cassandra git commit: Add support for type casting in selection clause

Repository: cassandra
Updated Branches:
  refs/heads/trunk 334fbad7b -> 269c5d4f8


Add support for type casting in selection clause

patch by Benjamin Lerer; reviewed by Robert Stupp for (CASSANDRA-10310)


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/269c5d4f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/269c5d4f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/269c5d4f

Branch: refs/heads/trunk
Commit: 269c5d4f85d6e4edd1a100533c9adb88a36ade70
Parents: 334fbad
Author: blerer <be...@datastax.com>
Authored: Mon Nov 16 15:04:51 2015 +0100
Committer: blerer <be...@datastax.com>
Committed: Mon Nov 16 15:08:09 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   9 +
 doc/cql3/CQL.textile                            |  39 ++-
 pylib/cqlshlib/cql3handling.py                  |   1 +
 src/java/org/apache/cassandra/cql3/Cql.g        |   6 +-
 .../cql3/functions/AbstractFunction.java        |  10 +
 .../cassandra/cql3/functions/AggregateFcts.java |  17 +-
 .../cassandra/cql3/functions/CastFcts.java      | 301 +++++++++++++++++++
 .../cassandra/cql3/functions/Function.java      |   8 +
 .../cql3/functions/FunctionResolver.java        |   4 +-
 .../cassandra/cql3/functions/TimeFcts.java      |  18 +-
 .../selection/AbstractFunctionSelector.java     |  11 +-
 .../cassandra/cql3/selection/Selectable.java    |  86 +++++-
 .../org/apache/cassandra/db/SystemKeyspace.java |   1 +
 .../cassandra/cql3/functions/CastFctsTest.java  | 295 ++++++++++++++++++
 .../validation/operations/AggregationTest.java  |   3 +-
 16 files changed, 766 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bf65c9b..c4a5b34 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.2
+ * Add support for type casting in selection clause (CASSANDRA-10310)
  * Added graphing option to cassandra-stress (CASSANDRA-7918)
  * Abort in-progress queries that time out (CASSANDRA-7392)
  * Add transparent data encryption core classes (CASSANDRA-9945)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index fdebbf2..eff96d8 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -13,6 +13,15 @@ restore snapshots created with the previous major version using the
 'sstableloader' tool. You can upgrade the file format of your snapshots
 using the provided 'sstableupgrade' tool.
 
+3.2
+===
+
+New features
+------------
+
+   - Support for type casting has been added to the selection clause.
+   
+   
 3.0
 ===
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index e55d8ca..c9c1f96 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1,6 +1,6 @@
 <link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">
 
-h1. Cassandra Query Language (CQL) v3.3.1
+h1. Cassandra Query Language (CQL) v3.3.2
 
 
 
@@ -1028,6 +1028,7 @@ bc(syntax)..
 <selector> ::= <identifier>
              | WRITETIME '(' <identifier> ')'
              | TTL '(' <identifier> ')'
+             | CAST '(' <selector> AS <type> ')'
              | <function> '(' (<selector> (',' <selector>)*)? ')'
 
 <where-clause> ::= <relation> ( AND <relation> )*
@@ -1069,7 +1070,7 @@ h4(#selectSelection). @<select-clause>@
 
 The @<select-clause>@ determines which columns needs to be queried and returned in the result-set. It consists of either the comma-separated list of <selector> or the wildcard character (@*@) to select all the columns defined for the table.
 
-A @<selector>@ is either a column name to retrieve or a @<function>@ of one or more @<term>@s. The function allowed are the same as for @<term>@ and are described in the "function section":#functions. In addition to these generic functions, the @WRITETIME@ (resp. @TTL@) function allows to select the timestamp of when the column was inserted (resp. the time to live (in seconds) for the column (or null if the column has no expiration set)).
+A @<selector>@ is either a column name to retrieve or a @<function>@ of one or more @<term>@s. The function allowed are the same as for @<term>@ and are described in the "function section":#functions. In addition to these generic functions, the @WRITETIME@ (resp. @TTL@) function allows to select the timestamp of when the column was inserted (resp. the time to live (in seconds) for the column (or null if the column has no expiration set)) and the "@CAST@":#castFun function can be used to convert one data type to another.
 
 Any @<selector>@ can be aliased using @AS@ keyword (see examples). Please note that @<where-clause>@ and @<order-by>@ clause should refer to the columns by their original names and not by their aliases.
 
@@ -1852,6 +1853,37 @@ h2(#functions). Functions
 
 CQL3 distinguishes between built-in functions (so called 'native functions') and "user-defined functions":#udfs.  CQL3 includes several native functions, described below:
 
+h3(#castFun). Cast
+
+The @cast@ function can be used to converts one native datatype to another.
+
+The following table describes the conversions supported by the @cast@ function. Cassandra will silently ignore any cast converting a datatype into its own datatype.
+
+|_. from    |_. to   |
+|@ascii@   |@text@, @varchar@                                                                                    |
+|@bigint@   |@tinyint@, @smallint@, @int@, @float@, @double@, @decimal@, @varint@, @text@, @varchar@             |
+|@boolean@  |@text@, @varchar@                                                                                   |
+|@counter@  |@tinyint@, @smallint@, @int@, @bigint@, @float@, @double@, @decimal@, @varint@, @text@, @varchar@   |
+|@date@      |@timestamp@                                                                                        |
+|@decimal@  |@tinyint@, @smallint@, @int@, @bigint@, @float@, @double@, @varint@, @text@, @varchar@              |
+|@double@   |@tinyint@, @smallint@, @int@, @bigint@, @float@, @decimal@, @varint@, @text@, @varchar@             |
+|@float@     |@tinyint@, @smallint@, @int@, @bigint@, @double@, @decimal@, @varint@, @text@, @varchar@           |
+|@inet@      |@text@, @varchar@                                                                                  |
+|@int@       |@tinyint@, @smallint@, @bigint@, @float@, @double@, @decimal@, @varint@, @text@, @varchar@         |
+|@smallint@ |@tinyint@, @int@, @bigint@, @float@, @double@, @decimal@, @varint@, @text@, @varchar@               |
+|@time@      |@text@, @varchar@                                                                                  |
+|@timestamp@|@date@, @text@, @varchar@                                                                           |
+|@timeuuid@ |@timestamp@, @date@, @text@, @varchar@                                                              |
+|@tinyint@  |@tinyint@, @smallint@, @int@, @bigint@, @float@, @double@, @decimal@, @varint@, @text@, @varchar@   |
+|@uuid@      |@text@, @varchar@                                                                                  |
+|@varint@   |@tinyint@, @smallint@, @int@, @bigint@, @float@, @double@, @decimal@, @text@, @varchar@             |
+
+
+The conversions rely strictly on Java's semantics. For example, the double value 1 will be converted to the text value '1.0'.
+
+bc(sample). 
+SELECT avg(cast(count as double)) FROM myTable
+
 h3(#tokenFun). Token
 
 The @token@ function allows to compute the token for a given partition key. The exact signature of the token function depends on the table concerned and of the partitioner used by the cluster.
@@ -2267,6 +2299,9 @@ h2(#changes). Changes
 
 The following describes the changes in each version of CQL.
 
+h3. 3.3.2
+
+* Adds @CAST@ functions. See "@Cast@":#castFun.
 
 h3. 3.3.1
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 42e542f..d446a35 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -686,6 +686,7 @@ syntax_rules += r'''
              | "WRITETIME" "(" [colname]=<cident> ")"
              | "TTL" "(" [colname]=<cident> ")"
              | "COUNT" "(" star=( "*" | "1" ) ")"
+             | "CAST" "(" <selector> "AS" <storageType> ")"
              | <functionName> <selectionFunctionArguments>
              ;
 <selectionFunctionArguments> ::= "(" ( <selector> ( "," <selector> )* )? ")"

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 50b1712..6b4eff6 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -328,9 +328,10 @@ selector returns [RawSelector s]
 unaliasedSelector returns [Selectable.Raw s]
     @init { Selectable.Raw tmp = null; }
     :  ( c=cident                                  { tmp = c; }
-       | K_COUNT '(' countArgument ')'             { tmp = new Selectable.WithFunction.Raw(FunctionName.nativeFunction("countRows"), Collections.<Selectable.Raw>emptyList());}
+       | K_COUNT '(' countArgument ')'             { tmp = Selectable.WithFunction.Raw.newCountRowsFunction();}
        | K_WRITETIME '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, true); }
        | K_TTL       '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, false); }
+       | K_CAST      '(' sn=unaliasedSelector K_AS t=native_type ')' {tmp = new Selectable.WithCast.Raw(sn, t);}
        | f=functionName args=selectionFunctionArgs { tmp = new Selectable.WithFunction.Raw(f, args); }
        ) ( '.' fi=cident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )* { $s = tmp; }
     ;
@@ -1574,7 +1575,7 @@ non_type_ident returns [ColumnIdentifier id]
 
 unreserved_keyword returns [String str]
     : u=unreserved_function_keyword     { $str = u; }
-    | k=(K_TTL | K_COUNT | K_WRITETIME | K_KEY) { $str = $k.text; }
+    | k=(K_TTL | K_COUNT | K_WRITETIME | K_KEY | K_CAST) { $str = $k.text; }
     ;
 
 unreserved_function_keyword returns [String str]
@@ -1674,6 +1675,7 @@ K_INTO:        I N T O;
 K_VALUES:      V A L U E S;
 K_TIMESTAMP:   T I M E S T A M P;
 K_TTL:         T T L;
+K_CAST:        C A S T;
 K_ALTER:       A L T E R;
 K_RENAME:      R E N A M E;
 K_ADD:         A D D;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
index b77f4d5..d36a492 100644
--- a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
@@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.cassandra.cql3.AssignmentTestable;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.commons.lang3.text.StrBuilder;
 
 /**
  * Base class for our native/hardcoded functions.
@@ -116,4 +117,13 @@ public abstract class AbstractFunction implements Function
         sb.append(") -> ").append(returnType.asCQL3Type());
         return sb.toString();
     }
+
+    @Override
+    public String columnName(List<String> columnNames)
+    {
+        return new StrBuilder(name().toString()).append('(')
+                                                .appendWithSeparators(columnNames, ", ")
+                                                .append(')')
+                                                .toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java b/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
index 7b5bdb8..c3f63fd 100644
--- a/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
+++ b/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
@@ -73,17 +73,6 @@ public abstract class AggregateFcts
     }
 
     /**
-     * Checks if the specified function is the count rows (e.g. COUNT(*) or COUNT(1)) function.
-     *
-     * @param function the function to check
-     * @return <code>true</code> if the specified function is the count rows one, <code>false</code> otherwise.
-     */
-    public static boolean isCountRows(Function function)
-    {
-        return function == countRowsFunction;
-    }
-
-    /**
      * The function used to count the number of rows of a result set. This function is called when COUNT(*) or COUNT(1)
      * is specified.
      */
@@ -112,6 +101,12 @@ public abstract class AggregateFcts
                         }
                     };
                 }
+
+                @Override
+                public String columnName(List<String> columnNames)
+                {
+                    return "count";
+                }
             };
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/functions/CastFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/CastFcts.java b/src/java/org/apache/cassandra/cql3/functions/CastFcts.java
new file mode 100644
index 0000000..7349f0c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/CastFcts.java
@@ -0,0 +1,301 @@
+/*
+ * 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.functions;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.FloatType;
+import org.apache.cassandra.db.marshal.InetAddressType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.ShortType;
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.apache.cassandra.db.marshal.TimeType;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.db.marshal.TimestampType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.commons.lang3.text.WordUtils;
+
+/**
+ * Casting functions
+ *
+ */
+public final class CastFcts
+{
+    private static final String FUNCTION_NAME_PREFIX = "castAs";
+
+    public static Collection<Function> all()
+    {
+        List<Function> functions = new ArrayList<>();
+
+        @SuppressWarnings("unchecked")
+        final AbstractType<? extends Number>[] numericTypes = new AbstractType[] {ByteType.instance,
+                                                                                  ShortType.instance,
+                                                                                  Int32Type.instance,
+                                                                                  LongType.instance,
+                                                                                  FloatType.instance,
+                                                                                  DoubleType.instance,
+                                                                                  DecimalType.instance,
+                                                                                  CounterColumnType.instance,
+                                                                                  IntegerType.instance};
+
+        for (AbstractType<? extends Number> inputType : numericTypes)
+        {
+            addFunctionIfNeeded(functions, inputType, ByteType.instance, Number::byteValue);
+            addFunctionIfNeeded(functions, inputType, ShortType.instance, Number::shortValue);
+            addFunctionIfNeeded(functions, inputType, Int32Type.instance, Number::intValue);
+            addFunctionIfNeeded(functions, inputType, LongType.instance, Number::longValue);
+            addFunctionIfNeeded(functions, inputType, FloatType.instance, Number::floatValue);
+            addFunctionIfNeeded(functions, inputType, DoubleType.instance, Number::doubleValue);
+            addFunctionIfNeeded(functions, inputType, DecimalType.instance, p -> BigDecimal.valueOf(p.doubleValue()));
+            addFunctionIfNeeded(functions, inputType, IntegerType.instance, p -> BigInteger.valueOf(p.longValue()));
+            functions.add(CastAsTextFunction.create(inputType, AsciiType.instance));
+            functions.add(CastAsTextFunction.create(inputType, UTF8Type.instance));
+        }
+
+        functions.add(JavaFunctionWrapper.create(AsciiType.instance, UTF8Type.instance, p -> p));
+
+        functions.add(CastAsTextFunction.create(InetAddressType.instance, AsciiType.instance));
+        functions.add(CastAsTextFunction.create(InetAddressType.instance, UTF8Type.instance));
+
+        functions.add(CastAsTextFunction.create(BooleanType.instance, AsciiType.instance));
+        functions.add(CastAsTextFunction.create(BooleanType.instance, UTF8Type.instance));
+
+        functions.add(CassandraFunctionWrapper.create(TimeUUIDType.instance, SimpleDateType.instance, TimeFcts.timeUuidtoDate));
+        functions.add(CassandraFunctionWrapper.create(TimeUUIDType.instance, TimestampType.instance, TimeFcts.timeUuidToTimestamp));
+        functions.add(CastAsTextFunction.create(TimeUUIDType.instance, AsciiType.instance));
+        functions.add(CastAsTextFunction.create(TimeUUIDType.instance, UTF8Type.instance));
+        functions.add(CassandraFunctionWrapper.create(TimestampType.instance, SimpleDateType.instance, TimeFcts.timestampToDate));
+        functions.add(CastAsTextFunction.create(TimestampType.instance, AsciiType.instance));
+        functions.add(CastAsTextFunction.create(TimestampType.instance, UTF8Type.instance));
+        functions.add(CassandraFunctionWrapper.create(SimpleDateType.instance, TimestampType.instance, TimeFcts.dateToTimestamp));
+        functions.add(CastAsTextFunction.create(SimpleDateType.instance, AsciiType.instance));
+        functions.add(CastAsTextFunction.create(SimpleDateType.instance, UTF8Type.instance));
+        functions.add(CastAsTextFunction.create(TimeType.instance, AsciiType.instance));
+        functions.add(CastAsTextFunction.create(TimeType.instance, UTF8Type.instance));
+
+        functions.add(CastAsTextFunction.create(UUIDType.instance, AsciiType.instance));
+        functions.add(CastAsTextFunction.create(UUIDType.instance, UTF8Type.instance));
+
+        return functions;
+    }
+
+    /**
+     * Creates the name of the cast function use to cast to the specified type.
+     *
+     * @param outputType the output type
+     * @return the name of the cast function use to cast to the specified type
+     */
+    public static String getFunctionName(AbstractType<?> outputType)
+    {
+        return getFunctionName(outputType.asCQL3Type());
+    }
+
+    /**
+     * Creates the name of the cast function use to cast to the specified type.
+     *
+     * @param outputType the output type
+     * @return the name of the cast function use to cast to the specified type
+     */
+    public static String getFunctionName(CQL3Type outputType)
+    {
+        return FUNCTION_NAME_PREFIX + WordUtils.capitalize(toLowerCaseString(outputType));
+    }
+
+    /**
+     * Adds to the list a function converting the input type in to the output type if they are not the same.
+     *
+     * @param functions the list to add to
+     * @param inputType the input type
+     * @param outputType the output type
+     * @param converter the function use to convert the input type into the output type
+     */
+    private static <I, O> void addFunctionIfNeeded(List<Function> functions,
+                                                   AbstractType<I> inputType,
+                                                   AbstractType<O> outputType,
+                                                   java.util.function.Function<I, O> converter)
+    {
+        if (!inputType.equals(outputType))
+            functions.add(JavaFunctionWrapper.create(inputType, outputType, converter));
+    }
+
+    private static String toLowerCaseString(CQL3Type type)
+    {
+        return type.toString().toLowerCase();
+    }
+
+    /**
+     * Base class for the CAST functions.
+     *
+     * @param <I> the input type
+     * @param <O> the output type
+     */
+    private static abstract class CastFunction<I, O> extends NativeScalarFunction
+    {
+        public CastFunction(AbstractType<I> inputType, AbstractType<O> outputType)
+        {
+            super(getFunctionName(outputType), outputType, inputType);
+        }
+
+        @Override
+        public String columnName(List<String> columnNames)
+        {
+            return String.format("cast(%s as %s)", columnNames.get(0), toLowerCaseString(outputType().asCQL3Type()));
+        }
+
+        @SuppressWarnings("unchecked")
+        protected AbstractType<O> outputType()
+        {
+            return (AbstractType<O>) returnType;
+        }
+
+        @SuppressWarnings("unchecked")
+        protected AbstractType<I> inputType()
+        {
+            return (AbstractType<I>) argTypes.get(0);
+        }
+    };
+
+    /**
+     * <code>CastFunction</code> that implements casting by wrapping a java <code>Function</code>.
+     *
+     * @param <I> the input parameter
+     * @param <O> the output parameter
+     */
+    private static final class JavaFunctionWrapper<I, O> extends CastFunction<I, O>
+    {
+        /**
+         * The java function used to convert the input type into the output one.
+         */
+        private final java.util.function.Function<I, O> converter;
+
+        public static <I, O> JavaFunctionWrapper<I, O> create(AbstractType<I> inputType,
+                                                              AbstractType<O> outputType,
+                                                              java.util.function.Function<I, O> converter)
+        {
+            return new JavaFunctionWrapper<I, O>(inputType, outputType, converter);
+        }
+
+        private JavaFunctionWrapper(AbstractType<I> inputType,
+                                    AbstractType<O> outputType,
+                                    java.util.function.Function<I, O> converter)
+        {
+            super(inputType, outputType);
+            this.converter = converter;
+        }
+
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return outputType().decompose(converter.apply(inputType().compose(bb)));
+        }
+    };
+
+    /**
+     * <code>CastFunction</code> that implements casting by wrapping an existing <code>NativeScalarFunction</code>.
+     *
+     * @param <I> the input parameter
+     * @param <O> the output parameter
+     */
+    private static final class CassandraFunctionWrapper<I, O> extends CastFunction<I, O>
+    {
+        /**
+         * The native scalar function used to perform the conversion.
+         */
+        private final NativeScalarFunction delegate;
+
+        public static <I, O> CassandraFunctionWrapper<I, O> create(AbstractType<I> inputType,
+                                                                   AbstractType<O> outputType,
+                                                                   NativeScalarFunction delegate)
+        {
+            return new CassandraFunctionWrapper<I, O>(inputType, outputType, delegate);
+        }
+
+        private CassandraFunctionWrapper(AbstractType<I> inputType,
+                                         AbstractType<O> outputType,
+                                         NativeScalarFunction delegate)
+        {
+            super(inputType, outputType);
+            assert delegate.argTypes().size() == 1 && inputType.equals(delegate.argTypes().get(0));
+            assert outputType.equals(delegate.returnType());
+            this.delegate = delegate;
+        }
+
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            return delegate.execute(protocolVersion, parameters);
+        }
+    };
+
+    /**
+     * <code>CastFunction</code> that can be used to cast a type into ascii or text types.
+     *
+     * @param <I> the input parameter
+     */
+    private static final class CastAsTextFunction<I> extends CastFunction<I, String>
+    {
+
+        public static <I> CastAsTextFunction<I> create(AbstractType<I> inputType,
+                                                       AbstractType<String> outputType)
+        {
+            return new CastAsTextFunction<I>(inputType, outputType);
+        }
+
+        private CastAsTextFunction(AbstractType<I> inputType,
+                                    AbstractType<String> outputType)
+        {
+            super(inputType, outputType);
+        }
+
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            StringBuilder builder = new StringBuilder();
+            inputType().getSerializer().toCQLLiteral(bb, builder);
+            return outputType().decompose(builder.toString());
+        }
+    };
+
+    /**
+     * The class must not be instantiated as it contains only static variables.
+     */
+    private CastFcts()
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/functions/Function.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Function.java b/src/java/org/apache/cassandra/cql3/functions/Function.java
index ed6e2a7..f50657a 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Function.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Function.java
@@ -47,4 +47,12 @@ public interface Function extends AssignmentTestable
     public Iterable<Function> getFunctions();
 
     public boolean hasReferenceTo(Function function);
+
+    /**
+     * Returns the name of the function to use within a ResultSet.
+     *
+     * @param columnNames the names of the columns used to call the function
+     * @return the name of the function to use within a ResultSet
+     */
+    public String columnName(List<String> columnNames);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java b/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
index be2daae..9e0b706 100644
--- a/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
@@ -126,9 +126,11 @@ public final class FunctionResolver
             }
         }
 
-        if (compatibles == null || compatibles.isEmpty())
+        if (compatibles == null)
+        {
             throw new InvalidRequestException(String.format("Invalid call to function %s, none of its type signatures match (known type signatures: %s)",
                                                             name, format(candidates)));
+        }
 
         if (compatibles.size() > 1)
             throw new InvalidRequestException(String.format("Ambiguous call to function %s (can be matched by following signatures: %s): use type casts to disambiguate",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
index 1b87261..623feba 100644
--- a/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
+++ b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
@@ -87,7 +87,7 @@ public abstract class TimeFcts
      * Function that convert a value of <code>TIMEUUID</code> into a value of type <code>TIMESTAMP</code>.
      * @deprecated Replaced by the {@link #timeUuidToTimestamp} function
      */
-    public static final Function dateOfFct = new NativeScalarFunction("dateof", TimestampType.instance, TimeUUIDType.instance)
+    public static final NativeScalarFunction dateOfFct = new NativeScalarFunction("dateof", TimestampType.instance, TimeUUIDType.instance)
     {
         private volatile boolean hasLoggedDeprecationWarning;
 
@@ -113,7 +113,7 @@ public abstract class TimeFcts
      * Function that convert a value of type <code>TIMEUUID</code> into an UNIX timestamp.
      * @deprecated Replaced by the {@link #timeUuidToUnixTimestamp} function
      */
-    public static final Function unixTimestampOfFct = new NativeScalarFunction("unixtimestampof", LongType.instance, TimeUUIDType.instance)
+    public static final NativeScalarFunction unixTimestampOfFct = new NativeScalarFunction("unixtimestampof", LongType.instance, TimeUUIDType.instance)
     {
         private volatile boolean hasLoggedDeprecationWarning;
 
@@ -137,7 +137,7 @@ public abstract class TimeFcts
     /**
      * Function that convert a value of <code>TIMEUUID</code> into a value of type <code>DATE</code>.
      */
-    public static final Function timeUuidtoDate = new NativeScalarFunction("todate", SimpleDateType.instance, TimeUUIDType.instance)
+    public static final NativeScalarFunction timeUuidtoDate = new NativeScalarFunction("todate", SimpleDateType.instance, TimeUUIDType.instance)
     {
         public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
         {
@@ -153,7 +153,7 @@ public abstract class TimeFcts
     /**
      * Function that convert a value of type <code>TIMEUUID</code> into a value of type <code>TIMESTAMP</code>.
      */
-    public static final Function timeUuidToTimestamp = new NativeScalarFunction("totimestamp", TimestampType.instance, TimeUUIDType.instance)
+    public static final NativeScalarFunction timeUuidToTimestamp = new NativeScalarFunction("totimestamp", TimestampType.instance, TimeUUIDType.instance)
     {
         public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
         {
@@ -169,7 +169,7 @@ public abstract class TimeFcts
     /**
      * Function that convert a value of type <code>TIMEUUID</code> into an UNIX timestamp.
      */
-    public static final Function timeUuidToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, TimeUUIDType.instance)
+    public static final NativeScalarFunction timeUuidToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, TimeUUIDType.instance)
     {
         public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
         {
@@ -184,7 +184,7 @@ public abstract class TimeFcts
     /**
      * Function that convert a value of type <code>TIMESTAMP</code> into an UNIX timestamp.
      */
-    public static final Function timestampToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, TimestampType.instance)
+    public static final NativeScalarFunction timestampToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, TimestampType.instance)
     {
         public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
         {
@@ -200,7 +200,7 @@ public abstract class TimeFcts
    /**
     * Function that convert a value of type <code>TIMESTAMP</code> into a <code>DATE</code>.
     */
-   public static final Function timestampToDate = new NativeScalarFunction("todate", SimpleDateType.instance, TimestampType.instance)
+   public static final NativeScalarFunction timestampToDate = new NativeScalarFunction("todate", SimpleDateType.instance, TimestampType.instance)
    {
        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
        {
@@ -216,7 +216,7 @@ public abstract class TimeFcts
    /**
     * Function that convert a value of type <code>TIMESTAMP</code> into a <code>DATE</code>.
     */
-   public static final Function dateToTimestamp = new NativeScalarFunction("totimestamp", TimestampType.instance, SimpleDateType.instance)
+   public static final NativeScalarFunction dateToTimestamp = new NativeScalarFunction("totimestamp", TimestampType.instance, SimpleDateType.instance)
    {
        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
        {
@@ -232,7 +232,7 @@ public abstract class TimeFcts
    /**
     * Function that convert a value of type <code>DATE</code> into an UNIX timestamp.
     */
-   public static final Function dateToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, SimpleDateType.instance)
+   public static final NativeScalarFunction dateToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, SimpleDateType.instance)
    {
        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
        {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index bf1234f..0ccff6b 100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@ -24,9 +24,6 @@ import java.util.List;
 import com.google.common.collect.Iterables;
 
 import org.apache.commons.lang3.text.StrBuilder;
-
-import org.apache.cassandra.cql3.functions.AggregateFcts;
-
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.functions.Function;
@@ -56,13 +53,7 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
         {
             protected String getColumnName()
             {
-                if (AggregateFcts.isCountRows(fun))
-                    return "count";
-
-                return new StrBuilder(fun.name().toString()).append('(')
-                                                            .appendWithSeparators(factories.getColumnNames(), ", ")
-                                                            .append(')')
-                                                            .toString();
+                return fun.columnName(factories.getColumnNames());
             }
 
             protected AbstractType<?> getReturnType()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 717fe7c..c3e0331 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -19,12 +19,13 @@
 package org.apache.cassandra.cql3.selection;
 
 import java.util.ArrayList;
+import java.util.Collections;
 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.CQL3Type;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -33,8 +34,7 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 
 public abstract class Selectable
 {
-    public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
-            throws InvalidRequestException;
+    public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs);
 
     protected static int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
     {
@@ -75,7 +75,7 @@ public abstract class Selectable
         }
 
         public Selector.Factory newSelectorFactory(CFMetaData cfm,
-                                                   List<ColumnDefinition> defs) throws InvalidRequestException
+                                                   List<ColumnDefinition> defs)
         {
             ColumnDefinition def = cfm.getColumnDefinition(id);
             if (def == null)
@@ -136,8 +136,7 @@ public abstract class Selectable
                                    .toString();
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm,
-                                                   List<ColumnDefinition> defs) throws InvalidRequestException
+        public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
         {
             SelectorFactories factories  =
                     SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, cfm, defs);
@@ -152,6 +151,7 @@ public abstract class Selectable
 
             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));
@@ -170,6 +170,12 @@ public abstract class Selectable
                 this.args = args;
             }
 
+            public static Raw newCountRowsFunction()
+            {
+                return new Raw(AggregateFcts.countRowsFunction.name(),
+                               Collections.emptyList());
+            }
+
             public WithFunction prepare(CFMetaData cfm)
             {
                 List<Selectable> preparedArgs = new ArrayList<>(args.size());
@@ -185,6 +191,71 @@ public abstract class Selectable
         }
     }
 
+    public static class WithCast extends Selectable
+    {
+        private final CQL3Type type;
+        private final Selectable arg;
+
+        public WithCast(Selectable arg, CQL3Type type)
+        {
+            this.arg = arg;
+            this.type = type;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("cast(%s as %s)", arg, type.toString().toLowerCase());
+        }
+
+        public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
+        {
+            SelectorFactories factories  =
+                    SelectorFactories.createFactoriesAndCollectColumnDefinitions(Collections.singletonList(arg), cfm, defs);
+
+            Selector.Factory factory = factories.get(0);
+
+            // If the user is trying to cast a type on its own type we simply ignore it.
+            if (type.getType().equals(factory.getReturnType()))
+            {
+                return factory;
+            }
+
+            FunctionName name = FunctionName.nativeFunction(CastFcts.getFunctionName(type));
+            Function fun = FunctionResolver.get(cfm.ksName, name, factories.newInstances(), cfm.ksName, cfm.cfName, null);
+
+            if (fun == null)
+            {
+                    throw new InvalidRequestException(String.format("%s cannot be cast to %s",
+                                                                    defs.get(0).name,
+                                                                    type));
+            }
+            return AbstractFunctionSelector.newFactory(fun, factories);
+        }
+
+        public static class Raw implements Selectable.Raw
+        {
+            private final CQL3Type type;
+            private final Selectable.Raw arg;
+
+            public Raw(Selectable.Raw arg, CQL3Type type)
+            {
+                this.arg = arg;
+                this.type = type;
+            }
+
+            public WithCast prepare(CFMetaData cfm)
+            {
+                return new WithCast(arg.prepare(cfm), type);
+            }
+
+            public boolean processesSelection()
+            {
+                return true;
+            }
+        }
+    }
+
     public static class WithFieldSelection extends Selectable
     {
         public final Selectable selected;
@@ -202,8 +273,7 @@ public abstract class Selectable
             return String.format("%s.%s", selected, field);
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm,
-                                                   List<ColumnDefinition> defs) throws InvalidRequestException
+        public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
         {
             Selector.Factory factory = selected.newSelectorFactory(cfm, defs);
             AbstractType<?> type = factory.newInstance().getType();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 83c8c1c..c0c1b86 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -450,6 +450,7 @@ public final class SystemKeyspace
                         .add(TimeFcts.all())
                         .add(BytesConversionFcts.all())
                         .add(AggregateFcts.all())
+                        .add(CastFcts.all())
                         .build();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/test/unit/org/apache/cassandra/cql3/functions/CastFctsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/functions/CastFctsTest.java b/test/unit/org/apache/cassandra/cql3/functions/CastFctsTest.java
new file mode 100644
index 0000000..ed30f03
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/functions/CastFctsTest.java
@@ -0,0 +1,295 @@
+/*
+ * 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.functions;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Date;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+import org.apache.cassandra.utils.UUIDGen;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+import org.junit.Test;
+
+public class CastFctsTest extends CQLTester
+{
+    @Test
+    public void testInvalidQueries() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b text, c double)");
+
+        assertInvalidSyntaxMessage("no viable alternative at input '(' (... b, c) VALUES ([CAST](...)",
+                                   "INSERT INTO %s (a, b, c) VALUES (CAST(? AS int), ?, ?)", 1.6, "test", 6.3);
+
+        assertInvalidSyntaxMessage("no viable alternative at input '(' (..." + KEYSPACE + "." + currentTable()
+                + " SET c = [cast](...)",
+                                   "UPDATE %s SET c = cast(? as double) WHERE a = ?", 1, 1);
+
+        assertInvalidSyntaxMessage("no viable alternative at input '(' (...= ? WHERE a = [CAST] (...)",
+                                   "UPDATE %s SET c = ? WHERE a = CAST (? AS INT)", 1, 2.0);
+
+        assertInvalidSyntaxMessage("no viable alternative at input '(' (..." + KEYSPACE + "." + currentTable()
+                + " WHERE a = [CAST] (...)",
+                                   "DELETE FROM %s WHERE a = CAST (? AS INT)", 1, 2.0);
+
+        assertInvalidSyntaxMessage("no viable alternative at input '(' (..." + KEYSPACE + "." + currentTable()
+                + " WHERE a = [CAST] (...)",
+                                   "SELECT * FROM %s WHERE a = CAST (? AS INT)", 1, 2.0);
+
+        assertInvalidMessage("a cannot be cast to boolean", "SELECT CAST(a AS boolean) FROM %s");
+    }
+
+    @Test
+    public void testNumericCastsInSelectionClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a tinyint primary key,"
+                                   + " b smallint,"
+                                   + " c int,"
+                                   + " d bigint,"
+                                   + " e float,"
+                                   + " f double,"
+                                   + " g decimal,"
+                                   + " h varint,"
+                                   + " i int)");
+
+        execute("INSERT INTO %s (a, b, c, d, e, f, g, h) VALUES (?, ?, ?, ?, ?, ?, ?, ?)",
+                (byte) 1, (short) 2, 3, 4L, 5.2F, 6.3, BigDecimal.valueOf(6.3), BigInteger.valueOf(4));
+
+        assertColumnNames(execute("SELECT CAST(b AS int), CAST(c AS int), CAST(d AS double) FROM %s"),
+                          "cast(b as int)",
+                          "c",
+                          "cast(d as double)");
+
+        assertRows(execute("SELECT CAST(a AS tinyint), " +
+                "CAST(b AS tinyint), " +
+                "CAST(c AS tinyint), " +
+                "CAST(d AS tinyint), " +
+                "CAST(e AS tinyint), " +
+                "CAST(f AS tinyint), " +
+                "CAST(g AS tinyint), " +
+                "CAST(h AS tinyint), " +
+                "CAST(i AS tinyint) FROM %s"),
+                   row((byte) 1, (byte) 2, (byte) 3, (byte) 4L, (byte) 5, (byte) 6, (byte) 6, (byte) 4, null));
+
+        assertRows(execute("SELECT CAST(a AS smallint), " +
+                "CAST(b AS smallint), " +
+                "CAST(c AS smallint), " +
+                "CAST(d AS smallint), " +
+                "CAST(e AS smallint), " +
+                "CAST(f AS smallint), " +
+                "CAST(g AS smallint), " +
+                "CAST(h AS smallint), " +
+                "CAST(i AS smallint) FROM %s"),
+                   row((short) 1, (short) 2, (short) 3, (short) 4L, (short) 5, (short) 6, (short) 6, (short) 4, null));
+
+        assertRows(execute("SELECT CAST(a AS int), " +
+                "CAST(b AS int), " +
+                "CAST(c AS int), " +
+                "CAST(d AS int), " +
+                "CAST(e AS int), " +
+                "CAST(f AS int), " +
+                "CAST(g AS int), " +
+                "CAST(h AS int), " +
+                "CAST(i AS int) FROM %s"),
+                   row(1, 2, 3, 4, 5, 6, 6, 4, null));
+
+        assertRows(execute("SELECT CAST(a AS bigint), " +
+                "CAST(b AS bigint), " +
+                "CAST(c AS bigint), " +
+                "CAST(d AS bigint), " +
+                "CAST(e AS bigint), " +
+                "CAST(f AS bigint), " +
+                "CAST(g AS bigint), " +
+                "CAST(h AS bigint), " +
+                "CAST(i AS bigint) FROM %s"),
+                   row(1L, 2L, 3L, 4L, 5L, 6L, 6L, 4L, null));
+
+        assertRows(execute("SELECT CAST(a AS float), " +
+                "CAST(b AS float), " +
+                "CAST(c AS float), " +
+                "CAST(d AS float), " +
+                "CAST(e AS float), " +
+                "CAST(f AS float), " +
+                "CAST(g AS float), " +
+                "CAST(h AS float), " +
+                "CAST(i AS float) FROM %s"),
+                   row(1.0F, 2.0F, 3.0F, 4.0F, 5.2F, 6.3F, 6.3F, 4.0F, null));
+
+        assertRows(execute("SELECT CAST(a AS double), " +
+                "CAST(b AS double), " +
+                "CAST(c AS double), " +
+                "CAST(d AS double), " +
+                "CAST(e AS double), " +
+                "CAST(f AS double), " +
+                "CAST(g AS double), " +
+                "CAST(h AS double), " +
+                "CAST(i AS double) FROM %s"),
+                   row(1.0, 2.0, 3.0, 4.0, (double) 5.2F, 6.3, 6.3, 4.0, null));
+
+        assertRows(execute("SELECT CAST(a AS decimal), " +
+                "CAST(b AS decimal), " +
+                "CAST(c AS decimal), " +
+                "CAST(d AS decimal), " +
+                "CAST(e AS decimal), " +
+                "CAST(f AS decimal), " +
+                "CAST(g AS decimal), " +
+                "CAST(h AS decimal), " +
+                "CAST(i AS decimal) FROM %s"),
+                   row(BigDecimal.valueOf(1.0),
+                       BigDecimal.valueOf(2.0),
+                       BigDecimal.valueOf(3.0),
+                       BigDecimal.valueOf(4.0),
+                       BigDecimal.valueOf(5.2F),
+                       BigDecimal.valueOf(6.3),
+                       BigDecimal.valueOf(6.3),
+                       BigDecimal.valueOf(4.0),
+                       null));
+
+        assertRows(execute("SELECT CAST(a AS ascii), " +
+                "CAST(b AS ascii), " +
+                "CAST(c AS ascii), " +
+                "CAST(d AS ascii), " +
+                "CAST(e AS ascii), " +
+                "CAST(f AS ascii), " +
+                "CAST(g AS ascii), " +
+                "CAST(h AS ascii), " +
+                "CAST(i AS ascii) FROM %s"),
+                   row("1",
+                       "2",
+                       "3",
+                       "4",
+                       "5.2",
+                       "6.3",
+                       "6.3",
+                       "4",
+                       null));
+
+        assertRows(execute("SELECT CAST(a AS text), " +
+                "CAST(b AS text), " +
+                "CAST(c AS text), " +
+                "CAST(d AS text), " +
+                "CAST(e AS text), " +
+                "CAST(f AS text), " +
+                "CAST(g AS text), " +
+                "CAST(h AS text), " +
+                "CAST(i AS text) FROM %s"),
+                   row("1",
+                       "2",
+                       "3",
+                       "4",
+                       "5.2",
+                       "6.3",
+                       "6.3",
+                       "4",
+                       null));
+    }
+
+    @Test
+    public void testTimeCastsInSelectionClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a timeuuid primary key, b timestamp, c date, d time)");
+
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        DateTime date = DateTimeFormat.forPattern("yyyy-MM-dd")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21");
+
+        long timeInMillis = dateTime.getMillis();
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, '2015-05-21 11:03:02+00', '2015-05-21', '11:03:02')",
+                UUIDGen.getTimeUUID(timeInMillis));
+
+        assertRows(execute("SELECT CAST(a AS timestamp), " +
+                           "CAST(b AS timestamp), " +
+                           "CAST(c AS timestamp) FROM %s"),
+                   row(new Date(dateTime.getMillis()), new Date(dateTime.getMillis()), new Date(date.getMillis())));
+
+        int timeInMillisToDay = SimpleDateSerializer.timeInMillisToDay(date.getMillis());
+        assertRows(execute("SELECT CAST(a AS date), " +
+                           "CAST(b AS date), " +
+                           "CAST(c AS date) FROM %s"),
+                   row(timeInMillisToDay, timeInMillisToDay, timeInMillisToDay));
+
+        assertRows(execute("SELECT CAST(b AS text), " +
+                           "CAST(c AS text), " +
+                           "CAST(d AS text) FROM %s"),
+                   row("2015-05-21T11:03:02.000Z", "2015-05-21", "11:03:02.000000000"));
+    }
+
+    @Test
+    public void testOtherTypeCastsInSelectionClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a ascii primary key,"
+                                   + " b inet,"
+                                   + " c boolean)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, '127.0.0.1', ?)",
+                "test", true);
+
+        assertRows(execute("SELECT CAST(a AS text), " +
+                "CAST(b AS text), " +
+                "CAST(c AS text) FROM %s"),
+                   row("test", "127.0.0.1", "true"));
+    }
+
+    @Test
+    public void testCastsWithReverseOrder() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int,"
+                                   + " b smallint,"
+                                   + " c double,"
+                                   + " primary key (a, b)) WITH CLUSTERING ORDER BY (b DESC);");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)",
+                1, (short) 2, 6.3);
+
+        assertRows(execute("SELECT CAST(a AS tinyint), " +
+                "CAST(b AS tinyint), " +
+                "CAST(c AS tinyint) FROM %s"),
+                   row((byte) 1, (byte) 2, (byte) 6));
+
+        assertRows(execute("SELECT CAST(CAST(a AS tinyint) AS smallint), " +
+                "CAST(CAST(b AS tinyint) AS smallint), " +
+                "CAST(CAST(c AS tinyint) AS smallint) FROM %s"),
+                   row((short) 1, (short) 2, (short) 6));
+
+        assertRows(execute("SELECT CAST(CAST(CAST(a AS tinyint) AS double) AS text), " +
+                "CAST(CAST(CAST(b AS tinyint) AS double) AS text), " +
+                "CAST(CAST(CAST(c AS tinyint) AS double) AS text) FROM %s"),
+                   row("1.0", "2.0", "6.0"));
+
+        String f = createFunction(KEYSPACE, "int",
+                                  "CREATE FUNCTION %s(val int) " +
+                                          "RETURNS NULL ON NULL INPUT " +
+                                          "RETURNS double " +
+                                          "LANGUAGE java " +
+                                          "AS 'return (double)val;'");
+
+        assertRows(execute("SELECT " + f + "(CAST(b AS int)) FROM %s"),
+                   row((double) 2));
+
+        assertRows(execute("SELECT CAST(" + f + "(CAST(b AS int)) AS text) FROM %s"),
+                   row("2.0"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269c5d4f/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
index 1a532ac..3ad39c0 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -96,6 +96,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT COUNT(b), count(c), count(e), count(f) FROM %s LIMIT 2"), row(4L, 3L, 3L, 3L));
         assertRows(execute("SELECT COUNT(b), count(c), count(e), count(f) FROM %s WHERE a = 1 LIMIT 2"),
                    row(4L, 3L, 3L, 3L));
+        assertRows(execute("SELECT AVG(CAST(b AS double)) FROM %s"), row(11.0/4));
     }
 
     @Test
@@ -1371,7 +1372,7 @@ public class AggregationTest extends CQLTester
                                              "INITCOND null");
 
         assertRows(execute("SELECT initcond FROM system_schema.aggregates WHERE keyspace_name=? AND aggregate_name=?", KEYSPACE, shortFunctionName(aggregation)),
-                   row(null));
+                   row((Object) null));
 
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(set(7, 8, 9)));