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 2016/11/21 17:55:50 UTC

[4/5] cassandra git commit: Add support for arithmetic operators

Add support for arithmetic operators

patch by Benjamin Lerer; reviewed by Sylvain Lebresne for CASSANDRA-11935


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

Branch: refs/heads/trunk
Commit: 8b3de2f4908c4651491b0f20b80f7bb96cff26ed
Parents: 075539a
Author: Benjamin Lerer <b....@gmail.com>
Authored: Mon Nov 21 18:04:42 2016 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Mon Nov 21 18:04:42 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 doc/source/cql/changes.rst                      |   4 +-
 doc/source/cql/definitions.rst                  |   4 +-
 doc/source/cql/index.rst                        |   1 +
 doc/source/cql/operators.rst                    |  57 ++
 pylib/cqlshlib/cql3handling.py                  |   2 +-
 src/antlr/Lexer.g                               |   6 +-
 src/antlr/Parser.g                              | 248 +++++--
 .../org/apache/cassandra/cql3/Constants.java    |  58 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |  85 ++-
 src/java/org/apache/cassandra/cql3/Maps.java    | 122 ++-
 src/java/org/apache/cassandra/cql3/Sets.java    |  95 ++-
 src/java/org/apache/cassandra/cql3/Tuples.java  | 147 +++-
 .../org/apache/cassandra/cql3/UserTypes.java    | 115 ++-
 .../cassandra/cql3/functions/FunctionCall.java  |  37 +-
 .../cql3/functions/FunctionResolver.java        |  91 ++-
 .../cassandra/cql3/functions/OperationFcts.java | 380 ++++++++++
 .../cql3/selection/CollectionFactory.java       |  91 +++
 .../cql3/selection/ForwardingFactory.java       |  90 +++
 .../cassandra/cql3/selection/ListSelector.java  | 104 +++
 .../cassandra/cql3/selection/MapSelector.java   | 195 +++++
 .../cql3/selection/ScalarFunctionSelector.java  |   9 -
 .../cassandra/cql3/selection/Selectable.java    | 647 +++++++++++++++-
 .../cassandra/cql3/selection/Selector.java      |  11 -
 .../cassandra/cql3/selection/SetSelector.java   | 106 +++
 .../cassandra/cql3/selection/TupleSelector.java | 101 +++
 .../cql3/selection/UserTypeSelector.java        | 177 +++++
 .../org/apache/cassandra/db/SystemKeyspace.java |   1 +
 .../cassandra/db/marshal/AbstractType.java      |  13 +-
 .../cassandra/db/marshal/BooleanType.java       |   2 +-
 .../apache/cassandra/db/marshal/ByteType.java   |  56 +-
 .../cassandra/db/marshal/CounterColumnType.java |  40 +-
 .../apache/cassandra/db/marshal/DateType.java   |   2 +-
 .../cassandra/db/marshal/DecimalType.java       |  76 +-
 .../apache/cassandra/db/marshal/DoubleType.java |  69 +-
 .../apache/cassandra/db/marshal/EmptyType.java  |   2 +-
 .../apache/cassandra/db/marshal/FloatType.java  |  61 +-
 .../apache/cassandra/db/marshal/Int32Type.java  |  48 +-
 .../cassandra/db/marshal/IntegerType.java       |  69 +-
 .../cassandra/db/marshal/LexicalUUIDType.java   |   2 +-
 .../apache/cassandra/db/marshal/LongType.java   |  52 +-
 .../apache/cassandra/db/marshal/NumberType.java | 223 ++++++
 .../cassandra/db/marshal/ReversedType.java      |   2 +-
 .../apache/cassandra/db/marshal/ShortType.java  |  51 +-
 .../cassandra/db/marshal/TimeUUIDType.java      |   2 +-
 .../cassandra/db/marshal/TimestampType.java     |   2 +-
 .../apache/cassandra/db/marshal/TupleType.java  |   5 +
 .../apache/cassandra/db/marshal/UUIDType.java   |   2 +-
 .../apache/cassandra/db/marshal/UserType.java   |   5 +
 .../exceptions/OperationExecutionException.java |  57 ++
 .../cassandra/serializers/ByteSerializer.java   |   4 +-
 .../apache/cassandra/utils/ByteBufferUtil.java  |  17 +
 .../org/apache/cassandra/cql3/CQLTester.java    |   4 +-
 .../cql3/functions/OperationFctsTest.java       | 744 +++++++++++++++++++
 .../selection/SelectionColumnMappingTest.java   |  94 +++
 .../cql3/selection/TermSelectionTest.java       | 386 +++++++++-
 .../cql3/validation/operations/SelectTest.java  |  10 +
 57 files changed, 4767 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 24641a6..1f1625c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,6 @@
+3.12
+ * Add support for arithmetic operators (CASSANDRA-11935)
+
 3.11
  * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/doc/source/cql/changes.rst
----------------------------------------------------------------------
diff --git a/doc/source/cql/changes.rst b/doc/source/cql/changes.rst
index 913bdb4..a33bb63 100644
--- a/doc/source/cql/changes.rst
+++ b/doc/source/cql/changes.rst
@@ -27,8 +27,8 @@ The following describes the changes in each version of CQL.
 - Adds a new ``duration `` :ref:`data types <data-types>` (:jira:`11873`).
 - Support for ``GROUP BY`` (:jira:`10707`).
 - Adds a ``DEFAULT UNSET`` option for ``INSERT JSON`` to ignore omitted columns (:jira:`11424`).
-- Allows ``null`` as a legal value for TTL on insert and update. It will be treated as equivalent to
-inserting a 0 (:jira:`12216`).
+- Allows ``null`` as a legal value for TTL on insert and update. It will be treated as equivalent to inserting a 0 (:jira:`12216`).
+- Adds support for arithmetic operators (:jira:`11935`)
 
 3.4.2
 ^^^^^

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/doc/source/cql/definitions.rst
----------------------------------------------------------------------
diff --git a/doc/source/cql/definitions.rst b/doc/source/cql/definitions.rst
index e54bcd7..cd548f5 100644
--- a/doc/source/cql/definitions.rst
+++ b/doc/source/cql/definitions.rst
@@ -119,9 +119,10 @@ Terms
 CQL has the notion of a *term*, which denotes the kind of values that CQL support. Terms are defined by:
 
 .. productionlist::
-   term: `constant` | `literal` | `function_call` | `type_hint` | `bind_marker`
+   term: `constant` | `literal` | `function_call` | `arithmetic_operation` | `type_hint` | `bind_marker`
    literal: `collection_literal` | `udt_literal` | `tuple_literal`
    function_call: `identifier` '(' [ `term` (',' `term`)* ] ')'
+   arithmetic_operation: '-' `term` | `term` ('+' | '-' | '*' | '/' | '%') `term`
    type_hint: '(' `cql_type` `)` term
    bind_marker: '?' | ':' `identifier`
 
@@ -132,6 +133,7 @@ A term is thus one of:
   (see the linked sections for details).
 - A function call: see :ref:`the section on functions <cql-functions>` for details on which :ref:`native function
   <native-functions>` exists and how to define your own :ref:`user-defined ones <udfs>`.
+- An arithmetic operation between terms. see :ref:`the section on arithmetic operations <arithmetic_operators>`
 - A *type hint*: see the :ref:`related section <type-hints>` for details.
 - A bind marker, which denotes a variable to be bound at execution time. See the section on :ref:`prepared-statements`
   for details. A bind marker can be either anonymous (``?``) or named (``:some_name``). The latter form provides a more

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/doc/source/cql/index.rst
----------------------------------------------------------------------
diff --git a/doc/source/cql/index.rst b/doc/source/cql/index.rst
index 00d90e4..6fa8135 100644
--- a/doc/source/cql/index.rst
+++ b/doc/source/cql/index.rst
@@ -38,6 +38,7 @@ thrift API (and earlier version 1 and 2 of CQL).
    mvs
    security
    functions
+   operators
    json
    triggers
    appendices

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/doc/source/cql/operators.rst
----------------------------------------------------------------------
diff --git a/doc/source/cql/operators.rst b/doc/source/cql/operators.rst
new file mode 100644
index 0000000..05f1c61
--- /dev/null
+++ b/doc/source/cql/operators.rst
@@ -0,0 +1,57 @@
+.. 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.
+
+.. highlight:: cql
+
+.. _arithmetic_operators:
+
+Arithmetic Operators
+--------------------
+
+CQL supports the following operators:
+
+=============== =======================================================================================================
+ Operator        Description
+=============== =======================================================================================================
+ \- (unary)       Negates operand
+ \+               Addition
+ \-               Substraction
+ \*               Multiplication
+ /               Division
+ %               Returns the remainder of a division
+=============== =======================================================================================================
+
+Arithmetic operations are only supported on numeric types or counters.
+
+The return type of the operation will be based on the operand types:
+
+============= =========== ========== ========== ========== ========== ========== ========== ========== ==========
+ left/right   tinyint      smallint   int        bigint     counter    float      double     varint     decimal
+============= =========== ========== ========== ========== ========== ========== ========== ========== ==========
+ **tinyint**   tinyint     smallint   int        bigint     bigint     float      double     varint     decimal
+ **smallint**  smallint    smallint   int        bigint     bigint     float      double     varint     decimal
+ **int**       int         int        int        bigint     bigint     float      double     varint     decimal
+ **bigint**    bigint      bigint     bigint     bigint     bigint     double     double     varint     decimal
+ **counter**   bigint      bigint     bigint     bigint     bigint     double     double     varint     decimal
+ **float**     float       float      float      double     double     float      double     decimal    decimal
+ **double**    double      double     double     double     double     double     double     decimal    decimal
+ **varint**    varint      varint     varint     decimal    decimal    decimal    decimal    decimal    decimal
+ **decimal**   decimal     decimal    decimal    decimal    decimal    decimal    decimal    decimal    decimal
+============= =========== ========== ========== ========== ========== ========== ========== ========== ==========
+
+``*``, ``/`` and ``%`` operators have a higher precedence level than ``+`` and ``-`` operator. By consequence,
+they will be evaluated before. If two operator in an expression have the same precedence level, they will be evaluated
+left to right based on their position in the expression.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index c628dcd..f81b19f 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -152,7 +152,7 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
 <colon> ::=         ":" ;
 <star> ::=          "*" ;
 <endtoken> ::=      ";" ;
-<op> ::=            /[-+=,().]/ ;
+<op> ::=            /[+-=%/,().]/ ;
 <cmp> ::=           /[<>!]=?/ ;
 <brackets> ::=      /[][{}]/ ;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/antlr/Lexer.g
----------------------------------------------------------------------
diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g
index 23cbed6..1685964 100644
--- a/src/antlr/Lexer.g
+++ b/src/antlr/Lexer.g
@@ -176,8 +176,10 @@ K_EXISTS:      E X I S T S;
 
 K_MAP:         M A P;
 K_LIST:        L I S T;
-K_NAN:         N A N;
-K_INFINITY:    I N F I N I T Y;
+K_POSITIVE_NAN: N A N;
+K_NEGATIVE_NAN: '-' N A N;
+K_POSITIVE_INFINITY:    I N F I N I T Y;
+K_NEGATIVE_INFINITY: '-' I N F I N I T Y;
 K_TUPLE:       T U P L E;
 
 K_TRIGGER:     T R I G G E R;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/antlr/Parser.g
----------------------------------------------------------------------
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index 3d06dc3..5fd6851 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -260,7 +260,6 @@ useStatement returns [UseStatement stmt]
  */
 selectStatement returns [SelectStatement.RawStatement expr]
     @init {
-        boolean isDistinct = false;
         Term.Raw limit = null;
         Term.Raw perPartitionLimit = null;
         Map<ColumnDefinition.Raw, Boolean> orderings = new LinkedHashMap<>();
@@ -269,8 +268,8 @@ selectStatement returns [SelectStatement.RawStatement expr]
         boolean isJson = false;
     }
     : K_SELECT
-      ( K_JSON { isJson = true; } )?
-      ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause )
+        // json is a valid column name. By consequence, we need to resolve the ambiguity for "json - json"
+      ( (K_JSON selectClause)=> K_JSON { isJson = true; } )? sclause=selectClause
       K_FROM cf=columnFamilyName
       ( K_WHERE wclause=whereClause )?
       ( K_GROUP K_BY groupByClause[groups] ( ',' groupByClause[groups] )* )?
@@ -281,15 +280,22 @@ selectStatement returns [SelectStatement.RawStatement expr]
       {
           SelectStatement.Parameters params = new SelectStatement.Parameters(orderings,
                                                                              groups,
-                                                                             isDistinct,
+                                                                             $sclause.isDistinct,
                                                                              allowFiltering,
                                                                              isJson);
           WhereClause where = wclause == null ? WhereClause.empty() : wclause.build();
-          $expr = new SelectStatement.RawStatement(cf, params, sclause, where, limit, perPartitionLimit);
+          $expr = new SelectStatement.RawStatement(cf, params, $sclause.selectors, where, limit, perPartitionLimit);
       }
     ;
 
-selectClause returns [List<RawSelector> expr]
+selectClause returns [boolean isDistinct, List<RawSelector> selectors]
+    @init{ $isDistinct = false; }
+    // distinct is a valid column name. By consequence, we need to resolve the ambiguity for "distinct - distinct"
+    : (K_DISTINCT selectors)=> K_DISTINCT s=selectors { $isDistinct = true; $selectors = s; }
+    | s=selectors { $selectors = s; }
+    ;
+
+selectors returns [List<RawSelector> expr]
     : t1=selector { $expr = new ArrayList<RawSelector>(); $expr.add(t1); } (',' tN=selector { $expr.add(tN); })*
     | '\*' { $expr = Collections.<RawSelector>emptyList();}
     ;
@@ -299,28 +305,117 @@ selector returns [RawSelector s]
     : us=unaliasedSelector (K_AS c=noncol_ident { alias = c; })? { $s = new RawSelector(us, alias); }
     ;
 
+unaliasedSelector returns [Selectable.Raw s]
+    : a=selectionAddition {$s = a;}
+    ;
+
+selectionAddition returns [Selectable.Raw s]
+    :   l=selectionMultiplication   {$s = l;}
+        ( '+' r=selectionMultiplication {$s = Selectable.WithFunction.Raw.newOperation('+', $s, r);}
+        | '-' r=selectionMultiplication {$s = Selectable.WithFunction.Raw.newOperation('-', $s, r);}
+        )*
+    ;
+
+selectionMultiplication returns [Selectable.Raw s]
+    :   l=selectionGroup   {$s = l;}
+        ( '\*' r=selectionGroup {$s = Selectable.WithFunction.Raw.newOperation('*', $s, r);}
+        | '/' r=selectionGroup {$s = Selectable.WithFunction.Raw.newOperation('/', $s, r);}
+        | '%' r=selectionGroup {$s = Selectable.WithFunction.Raw.newOperation('\%', $s, r);}
+        )*
+    ;
+
+selectionGroup returns [Selectable.Raw s]
+    : (selectionGroupWithField)=>  f=selectionGroupWithField { $s=f; }
+    | g=selectionGroupWithoutField { $s=g; }
+    | '-' g=selectionGroup {$s = Selectable.WithFunction.Raw.newNegation(g);}
+    ;
+
+selectionGroupWithField returns [Selectable.Raw s]
+    @init { Selectable.Raw tmp = null; }
+    @after { $s = tmp; }
+    : g=selectionGroupWithoutField {tmp=g;} ( '.' fi=fident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )+
+    ;
+
+selectionGroupWithoutField returns [Selectable.Raw s]
+    @init { Selectable.Raw tmp = null; }
+    @after { $s = tmp; }
+    : sn=simpleUnaliasedSelector  { tmp=sn; }
+    | (selectionTypeHint)=> h=selectionTypeHint { tmp=h; }
+    | t=selectionTupleOrNestedSelector { tmp=t; }
+    | l=selectionList { tmp=l; }
+    | m=selectionMapOrSet { tmp=m; }
+    // UDTs are equivalent to maps from the syntax point of view, so the final decision will be done in Selectable.WithMapOrUdt
+    ;
+
+selectionTypeHint returns [Selectable.Raw s]
+    : '(' ct=comparatorType ')' a=selectionGroupWithoutField { $s = new Selectable.WithTypeHint.Raw(ct, a); }
+    ;
+
+selectionList returns [Selectable.Raw s]
+    @init { List<Selectable.Raw> l = new ArrayList<>(); }
+    @after { $s = new Selectable.WithList.Raw(l); }
+    : '[' ( t1=unaliasedSelector { l.add(t1); } ( ',' tn=unaliasedSelector { l.add(tn); } )* )? ']'
+    ;
+
+selectionMapOrSet returns [Selectable.Raw s]
+    : '{' t1=unaliasedSelector ( m=selectionMap[t1] { $s = m; } | st=selectionSet[t1] { $s = st; }) '}'
+    | '{' '}' { $s = new Selectable.WithSet.Raw(Collections.emptyList());}
+    ;
+
+selectionMap [Selectable.Raw k1] returns [Selectable.Raw s]
+    @init { List<Pair<Selectable.Raw, Selectable.Raw>> m = new ArrayList<>(); }
+    @after { $s = new Selectable.WithMapOrUdt.Raw(m); }
+      : ':' v1=unaliasedSelector   { m.add(Pair.create(k1, v1)); } ( ',' kn=unaliasedSelector ':' vn=unaliasedSelector { m.add(Pair.create(kn, vn)); } )*
+      ;
+
+selectionSet [Selectable.Raw t1] returns [Selectable.Raw s]
+    @init { List<Selectable.Raw> l = new ArrayList<>(); l.add(t1); }
+    @after { $s = new Selectable.WithSet.Raw(l); }
+      : ( ',' tn=unaliasedSelector { l.add(tn); } )*
+      ;
+
+selectionTupleOrNestedSelector returns [Selectable.Raw s]
+    @init { List<Selectable.Raw> l = new ArrayList<>(); }
+    @after { $s = new Selectable.BetweenParenthesesOrWithTuple.Raw(l); }
+    : '(' t1=unaliasedSelector { l.add(t1); } (',' tn=unaliasedSelector { l.add(tn); } )* ')'
+    ;
+
 /*
  * A single selection. The core of it is selecting a column, but we also allow any term and function, as well as
  * sub-element selection for UDT.
  */
-unaliasedSelector returns [Selectable.Raw s]
-    @init { Selectable.Raw tmp = null; }
-    :  ( c=cident                                  { tmp = c; }
-       | v=value                                   { tmp = new Selectable.WithTerm.Raw(v); }
-       | '(' ct=comparatorType ')' v=value         { tmp = new Selectable.WithTerm.Raw(new TypeCast(ct, v)); }
-       | K_COUNT '(' '\*' ')'                      { 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=fident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )* { $s = tmp; }
+simpleUnaliasedSelector returns [Selectable.Raw s]
+    : c=sident                                   { $s = c; }
+    | l=selectionLiteral                         { $s = new Selectable.WithTerm.Raw(l); }
+    | f=selectionFunction                        { $s = f; }
+    ;
+
+selectionFunction returns [Selectable.Raw s]
+    : K_COUNT '(' '\*' ')'                      { $s = Selectable.WithFunction.Raw.newCountRowsFunction(); }
+    | K_WRITETIME '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL.Raw(c, true); }
+    | K_TTL       '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL.Raw(c, false); }
+    | K_CAST      '(' sn=unaliasedSelector K_AS t=native_type ')' {$s = new Selectable.WithCast.Raw(sn, t);}
+    | f=functionName args=selectionFunctionArgs { $s = new Selectable.WithFunction.Raw(f, args); }
+    ;
+
+selectionLiteral returns [Term.Raw value]
+    : c=constant                     { $value = c; }
+    | K_NULL                         { $value = Constants.NULL_LITERAL; }
+    | ':' id=noncol_ident            { $value = newBindVariables(id); }
+    | QMARK                          { $value = newBindVariables(null); }
     ;
 
 selectionFunctionArgs returns [List<Selectable.Raw> a]
-    : '(' ')' { $a = Collections.emptyList(); }
-    | '(' s1=unaliasedSelector { List<Selectable.Raw> args = new ArrayList<Selectable.Raw>(); args.add(s1); }
-          ( ',' sn=unaliasedSelector { args.add(sn); } )*
-      ')' { $a = args; }
+    @init{ $a = new ArrayList<>(); }
+    : '(' (s1=unaliasedSelector { $a.add(s1); }
+          ( ',' sn=unaliasedSelector { $a.add(sn); } )*)?
+      ')'
+    ;
+
+sident returns [Selectable.Raw id]
+    : t=IDENT              { $id = Selectable.RawIdentifier.forUnquoted($t.text); }
+    | t=QUOTED_NAME        { $id = ColumnDefinition.RawIdentifier.forQuoted($t.text); }
+    | k=unreserved_keyword { $id = ColumnDefinition.RawIdentifier.forUnquoted(k); }
     ;
 
 whereClause returns [WhereClause.Builder clause]
@@ -661,14 +756,17 @@ cfamDefinition[CreateTableStatement.RawStatement expr]
     ;
 
 cfamColumns[CreateTableStatement.RawStatement expr]
-    : k=ident v=comparatorType { boolean isStatic=false; } (K_STATIC {isStatic = true;})? { $expr.addDefinition(k, v, isStatic); }
+    @init { boolean isStatic = false; }
+    : k=ident v=comparatorType (K_STATIC {isStatic = true;})? { $expr.addDefinition(k, v, isStatic); }
         (K_PRIMARY K_KEY { $expr.addKeyAliases(Collections.singletonList(k)); })?
     | K_PRIMARY K_KEY '(' pkDef[expr] (',' c=ident { $expr.addColumnAlias(c); } )* ')'
     ;
 
 pkDef[CreateTableStatement.RawStatement expr]
-    : k=ident { $expr.addKeyAliases(Collections.singletonList(k)); }
-    | '(' { List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>(); } k1=ident { l.add(k1); } ( ',' kn=ident { l.add(kn); } )* ')' { $expr.addKeyAliases(l); }
+    @init {List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>();}
+    @after{ $expr.addKeyAliases(l); }
+    : k1=ident { l.add(k1);}
+    | '(' k1=ident { l.add(k1); } ( ',' kn=ident { l.add(kn); } )* ')'
     ;
 
 cfamProperty[CFProperties props]
@@ -743,7 +841,7 @@ createMaterializedViewStatement returns [CreateViewStatement expr]
         List<ColumnDefinition.Raw> compositeKeys = new ArrayList<>();
     }
     : K_CREATE K_MATERIALIZED K_VIEW (K_IF K_NOT K_EXISTS { ifNotExists = true; })? cf=columnFamilyName K_AS
-        K_SELECT sclause=selectClause K_FROM basecf=columnFamilyName
+        K_SELECT sclause=selectors K_FROM basecf=columnFamilyName
         (K_WHERE wclause=whereClause)?
         K_PRIMARY K_KEY (
         '(' '(' k1=cident { partitionKeys.add(k1); } ( ',' kn=cident { partitionKeys.add(kn); } )* ')' ( ',' c1=cident { compositeKeys.add(c1); } )* ')'
@@ -849,14 +947,14 @@ alterTypeStatement returns [AlterTypeStatement expr]
     : K_ALTER K_TYPE name=userTypeName
           ( K_ALTER f=fident K_TYPE v=comparatorType { $expr = AlterTypeStatement.alter(name, f, v); }
           | K_ADD   f=fident v=comparatorType        { $expr = AlterTypeStatement.addition(name, f, v); }
-          | K_RENAME
-               { Map<FieldIdentifier, FieldIdentifier> renames = new HashMap<>(); }
-                 id1=fident K_TO toId1=fident { renames.put(id1, toId1); }
-                 ( K_AND idn=fident K_TO toIdn=fident { renames.put(idn, toIdn); } )*
-               { $expr = AlterTypeStatement.renames(name, renames); }
+          | K_RENAME r=renamedColumns                { $expr = AlterTypeStatement.renames(name, r); }
           )
     ;
 
+renamedColumns returns [Map<FieldIdentifier, FieldIdentifier> renames]
+    @init {$renames = new HashMap<>();}
+    : id1=fident K_TO toId1=fident { renames.put(id1, toId1); } ( K_AND idn=fident K_TO toIdn=fident { renames.put(idn, toIdn); } )*
+    ;
 
 /**
  * DROP KEYSPACE [IF EXISTS] <KSP>;
@@ -1155,7 +1253,7 @@ roleOptions[RoleOptions opts]
 
 roleOption[RoleOptions opts]
     :  K_PASSWORD '=' v=STRING_LITERAL { opts.setOption(IRoleManager.Option.PASSWORD, $v.text); }
-    |  K_OPTIONS '=' m=mapLiteral { opts.setOption(IRoleManager.Option.OPTIONS, convertPropertyMap(m)); }
+    |  K_OPTIONS '=' m=fullMapLiteral { opts.setOption(IRoleManager.Option.OPTIONS, convertPropertyMap(m)); }
     |  K_SUPERUSER '=' b=BOOLEAN { opts.setOption(IRoleManager.Option.SUPERUSER, Boolean.valueOf($b.text)); }
     |  K_LOGIN '=' b=BOOLEAN { opts.setOption(IRoleManager.Option.LOGIN, Boolean.valueOf($b.text)); }
     ;
@@ -1258,34 +1356,49 @@ constant returns [Constants.Literal constant]
     | t=DURATION       { $constant = Constants.Literal.duration($t.text);}
     | t=UUID           { $constant = Constants.Literal.uuid($t.text); }
     | t=HEXNUMBER      { $constant = Constants.Literal.hex($t.text); }
-    | { String sign=""; } ('-' {sign = "-"; } )? t=(K_NAN | K_INFINITY) { $constant = Constants.Literal.floatingPoint(sign + $t.text); }
+    | ((K_POSITIVE_NAN | K_NEGATIVE_NAN) { $constant = Constants.Literal.floatingPoint("NaN"); }
+        | K_POSITIVE_INFINITY  { $constant = Constants.Literal.floatingPoint("Infinity"); }
+        | K_NEGATIVE_INFINITY { $constant = Constants.Literal.floatingPoint("-Infinity"); })
     ;
 
-mapLiteral returns [Maps.Literal map]
-    : '{' { List<Pair<Term.Raw, Term.Raw>> m = new ArrayList<Pair<Term.Raw, Term.Raw>>(); }
-          ( k1=term ':' v1=term { m.add(Pair.create(k1, v1)); } ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )* )?
-      '}' { $map = new Maps.Literal(m); }
+fullMapLiteral returns [Maps.Literal map]
+    @init { List<Pair<Term.Raw, Term.Raw>> m = new ArrayList<Pair<Term.Raw, Term.Raw>>();}
+    @after{ $map = new Maps.Literal(m); }
+    : '{' ( k1=term ':' v1=term { m.add(Pair.create(k1, v1)); } ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )* )?
+      '}'
     ;
 
 setOrMapLiteral[Term.Raw t] returns [Term.Raw value]
-    : ':' v=term { List<Pair<Term.Raw, Term.Raw>> m = new ArrayList<Pair<Term.Raw, Term.Raw>>(); m.add(Pair.create(t, v)); }
-          ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )*
-      { $value = new Maps.Literal(m); }
-    | { List<Term.Raw> s = new ArrayList<Term.Raw>(); s.add(t); }
-          ( ',' tn=term { s.add(tn); } )*
-      { $value = new Sets.Literal(s); }
+    : m=mapLiteral[t] { $value=m; }
+    | s=setLiteral[t] { $value=s; }
+    ;
+
+setLiteral[Term.Raw t] returns [Term.Raw value]
+    @init { List<Term.Raw> s = new ArrayList<Term.Raw>(); s.add(t); }
+    @after { $value = new Sets.Literal(s); }
+    : ( ',' tn=term { s.add(tn); } )*
+    ;
+
+mapLiteral[Term.Raw k] returns [Term.Raw value]
+    @init { List<Pair<Term.Raw, Term.Raw>> m = new ArrayList<Pair<Term.Raw, Term.Raw>>(); }
+    @after { $value = new Maps.Literal(m); }
+    : ':' v=term {  m.add(Pair.create(k, v)); } ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )*
     ;
 
 collectionLiteral returns [Term.Raw value]
-    : '[' { List<Term.Raw> l = new ArrayList<Term.Raw>(); }
-          ( t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* )?
-      ']' { $value = new Lists.Literal(l); }
+    : l=listLiteral { $value = l; }
     | '{' t=term v=setOrMapLiteral[t] { $value = v; } '}'
     // Note that we have an ambiguity between maps and set for "{}". So we force it to a set literal,
     // and deal with it later based on the type of the column (SetLiteral.java).
     | '{' '}' { $value = new Sets.Literal(Collections.<Term.Raw>emptyList()); }
     ;
 
+listLiteral returns [Term.Raw value]
+    @init {List<Term.Raw> l = new ArrayList<Term.Raw>();}
+    @after {$value = new Lists.Literal(l);}
+    : '[' ( t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* )? ']' { $value = new Lists.Literal(l); }
+    ;
+
 usertypeLiteral returns [UserTypes.Literal ut]
     @init{ Map<FieldIdentifier, Term.Raw> m = new HashMap<>(); }
     @after{ $ut = new UserTypes.Literal(m); }
@@ -1338,9 +1451,33 @@ functionArgs returns [List<Term.Raw> args]
     ;
 
 term returns [Term.Raw term]
-    : v=value                          { $term = v; }
-    | f=function                       { $term = f; }
-    | '(' c=comparatorType ')' t=term  { $term = new TypeCast(c, t); }
+    : t=termAddition                          { $term = t; }
+    ;
+
+termAddition returns [Term.Raw term]
+    :   l=termMultiplication   {$term = l;}
+        ( '+' r=termMultiplication {$term = FunctionCall.Raw.newOperation('+', $term, r);}
+        | '-' r=termMultiplication {$term = FunctionCall.Raw.newOperation('-', $term, r);}
+        )*
+    ;
+
+termMultiplication returns [Term.Raw term]
+    :   l=termGroup   {$term = l;}
+        ( '\*' r=termGroup {$term = FunctionCall.Raw.newOperation('*', $term, r);}
+        | '/' r=termGroup {$term = FunctionCall.Raw.newOperation('/', $term, r);}
+        | '%' r=termGroup {$term = FunctionCall.Raw.newOperation('\%', $term, r);}
+        )*
+    ;
+
+termGroup returns [Term.Raw term]
+    : t=simpleTerm              { $term = t; }
+    | '-'  t=simpleTerm         { $term = FunctionCall.Raw.newNegation(t); }
+    ;
+
+simpleTerm returns [Term.Raw term]
+    : v=value                                 { $term = v; }
+    | f=function                              { $term = f; }
+    | '(' c=comparatorType ')' t=simpleTerm   { $term = new TypeCast(c, t); }
     ;
 
 columnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations]
@@ -1436,7 +1573,7 @@ properties[PropertyDefinitions props]
 
 property[PropertyDefinitions props]
     : k=noncol_ident '=' simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
-    | k=noncol_ident '=' map=mapLiteral { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
+    | k=noncol_ident '=' map=fullMapLiteral { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
     ;
 
 propertyValue returns [String str]
@@ -1463,8 +1600,7 @@ relation[WhereClause.Builder clauses]
         { $clauses.add(new SingleColumnRelation(name, Operator.IN, marker)); }
     | name=cident K_IN inValues=singleColumnInValues
         { $clauses.add(SingleColumnRelation.createInRelation($name.id, inValues)); }
-    | name=cident K_CONTAINS { Operator rt = Operator.CONTAINS; } (K_KEY { rt = Operator.CONTAINS_KEY; })?
-        t=term { $clauses.add(new SingleColumnRelation(name, rt, t)); }
+    | name=cident rt=containsOperator t=term { $clauses.add(new SingleColumnRelation(name, rt, t)); }
     | name=cident '[' key=term ']' type=relationType t=term { $clauses.add(new SingleColumnRelation(name, key, type, t)); }
     | ids=tupleOfIdentifiers
       ( K_IN
@@ -1489,6 +1625,10 @@ relation[WhereClause.Builder clauses]
     | '(' relation[$clauses] ')'
     ;
 
+containsOperator returns [Operator o]
+    : K_CONTAINS { o = Operator.CONTAINS; } (K_KEY { o = Operator.CONTAINS_KEY; })?
+    ;
+
 inMarker returns [AbstractMarker.INRaw marker]
     : QMARK { $marker = newINBindVariables(null); }
     | ':' name=noncol_ident { $marker = newINBindVariables(name); }
@@ -1587,9 +1727,9 @@ collection_type returns [CQL3Type.Raw pt]
     ;
 
 tuple_type returns [CQL3Type.Raw t]
-    : K_TUPLE '<' { List<CQL3Type.Raw> types = new ArrayList<>(); }
-         t1=comparatorType { types.add(t1); } (',' tn=comparatorType { types.add(tn); })*
-      '>' { $t = CQL3Type.Raw.tuple(types); }
+    @init {List<CQL3Type.Raw> types = new ArrayList<>();}
+    @after {$t = CQL3Type.Raw.tuple(types);}
+    : K_TUPLE '<' t1=comparatorType { types.add(t1); } (',' tn=comparatorType { types.add(tn); })* '>'
     ;
 
 username

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index 03e2053..8d54063 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
 
 import org.slf4j.Logger;
@@ -38,7 +40,54 @@ public abstract class Constants
 
     public enum Type
     {
-        STRING, INTEGER, UUID, FLOAT, BOOLEAN, HEX, DURATION;
+        STRING,
+        INTEGER
+        {
+            public AbstractType<?> getPreferedTypeFor(String text)
+            {
+                // We only try to determine the smallest possible type between int, long and BigInteger
+                BigInteger b = new BigInteger(text);
+
+                if (b.equals(BigInteger.valueOf(b.intValue())))
+                    return Int32Type.instance;
+
+                if (b.equals(BigInteger.valueOf(b.longValue())))
+                    return LongType.instance;
+
+                return IntegerType.instance;
+            }
+        },
+        UUID,
+        FLOAT
+        {
+            public AbstractType<?> getPreferedTypeFor(String text)
+            {
+                if ("NaN".equals(text) || "-NaN".equals(text) || "Infinity".equals(text) || "-Infinity".equals(text))
+                    return DoubleType.instance;
+
+                // We only try to determine the smallest possible type between double and BigDecimal
+                BigDecimal b = new BigDecimal(text);
+
+                if (b.equals(BigDecimal.valueOf(b.doubleValue())))
+                    return DoubleType.instance;
+
+                return DecimalType.instance;
+            }
+        },
+        BOOLEAN,
+        HEX,
+        DURATION;
+
+        /**
+         * Returns the exact type for the specified text
+         *
+         * @param text the text for which the type must be determined
+         * @return the exact type or {@code null} if it is not known.
+         */
+        public AbstractType<?> getPreferedTypeFor(String text)
+        {
+            return null;
+        }
     }
 
     private static class UnsetLiteral extends Term.Raw
@@ -119,12 +168,14 @@ public abstract class Constants
     {
         private final Type type;
         private final String text;
+        private final AbstractType<?> preferedType;
 
         private Literal(Type type, String text)
         {
             assert type != null && text != null;
             this.type = type;
             this.text = text;
+            this.preferedType = type.getPreferedTypeFor(text);
         }
 
         public static Literal string(String text)
@@ -204,6 +255,11 @@ public abstract class Constants
                 return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
 
             CQL3Type.Native nt = (CQL3Type.Native)receiverType;
+
+            // If the receiver type match the prefered type we can straight away return an exact match
+            if (nt.getType().equals(preferedType))
+                return AssignmentTestable.TestResult.EXACT_MATCH;
+
             switch (type)
             {
                 case STRING:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index 037162b..eb4b685 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -22,8 +22,11 @@ import static org.apache.cassandra.cql3.Constants.UNSET_VALUE;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
@@ -53,7 +56,67 @@ public abstract class Lists
 
     public static ColumnSpecification valueSpecOf(ColumnSpecification column)
     {
-        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((ListType)column.type).getElementsType());
+        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((ListType<?>)column.type).getElementsType());
+    }
+
+    /**
+     * Tests that the list with the specified elements can be assigned to the specified column.
+     *
+     * @param receiver the receiving column
+     * @param elements the list elements
+     */
+    public static AssignmentTestable.TestResult testListAssignment(ColumnSpecification receiver,
+                                                                   List<? extends AssignmentTestable> elements)
+    {
+        if (!(receiver.type instanceof ListType))
+            return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+
+        // If there is no elements, we can't say it's an exact match (an empty list if fundamentally polymorphic).
+        if (elements.isEmpty())
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+        ColumnSpecification valueSpec = valueSpecOf(receiver);
+        return AssignmentTestable.TestResult.testAll(receiver.ksName, valueSpec, elements);
+    }
+
+    /**
+     * Create a <code>String</code> representation of the list containing the specified elements.
+     *
+     * @param elements the list elements
+     * @return a <code>String</code> representation of the list
+     */
+    public static String listToString(List<?> elements)
+    {
+        return listToString(elements, Object::toString);
+    }
+
+    /**
+     * Create a <code>String</code> representation of the list from the specified items associated to
+     * the list elements.
+     *
+     * @param items items associated to the list elements
+     * @param mapper the mapper used to map the items to the <code>String</code> representation of the list elements
+     * @return a <code>String</code> representation of the list
+     */
+    public static <T> String listToString(Iterable<T> items, java.util.function.Function<T, String> mapper)
+    {
+        return StreamSupport.stream(items.spliterator(), false)
+                            .map(e -> mapper.apply(e))
+                            .collect(Collectors.joining(", ", "[", "]"));
+    }
+
+    /**
+     * Returns the exact ListType from the items if it can be known.
+     *
+     * @param items the items mapped to the list elements
+     * @param mapper the mapper used to retrieve the element types from the items
+     * @return the exact ListType from the items if it can be known or <code>null</code>
+     */
+    public static <T> AbstractType<?> getExactListTypeIfKnown(List<T> items,
+                                                              java.util.function.Function<T, AbstractType<?>> mapper)
+    {
+        Optional<AbstractType<?>> type = items.stream().map(mapper).filter(Objects::nonNull).findFirst();
+        return type.isPresent() ? ListType.getInstance(type.get(), false) : null;
     }
 
     public static class Literal extends Term.Raw
@@ -103,32 +166,18 @@ public abstract class Lists
 
         public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            if (!(receiver.type instanceof ListType))
-                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
-
-            // If there is no elements, we can't say it's an exact match (an empty list if fundamentally polymorphic).
-            if (elements.isEmpty())
-                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
-
-            ColumnSpecification valueSpec = Lists.valueSpecOf(receiver);
-            return AssignmentTestable.TestResult.testAll(keyspace, valueSpec, elements);
+            return testListAssignment(receiver, elements);
         }
 
         @Override
         public AbstractType<?> getExactTypeIfKnown(String keyspace)
         {
-            for (Term.Raw term : elements)
-            {
-                AbstractType<?> type = term.getExactTypeIfKnown(keyspace);
-                if (type != null)
-                    return ListType.getInstance(type, false);
-            }
-            return null;
+            return getExactListTypeIfKnown(elements, p -> p.getExactTypeIfKnown(keyspace));
         }
 
         public String getText()
         {
-            return elements.stream().map(Term.Raw::getText).collect(Collectors.joining(", ", "[", "]"));
+            return listToString(elements, Term.Raw::getText);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/Maps.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java
index 5c3bc44..6189285 100644
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@ -44,12 +44,89 @@ public abstract class Maps
 
     public static ColumnSpecification keySpecOf(ColumnSpecification column)
     {
-        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("key(" + column.name + ")", true), ((MapType)column.type).getKeysType());
+        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("key(" + column.name + ")", true), ((MapType<? , ?>)column.type).getKeysType());
     }
 
     public static ColumnSpecification valueSpecOf(ColumnSpecification column)
     {
-        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((MapType)column.type).getValuesType());
+        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((MapType<?, ?>)column.type).getValuesType());
+    }
+
+    /**
+     * Tests that the map with the specified entries can be assigned to the specified column.
+     *
+     * @param receiver the receiving column
+     * @param entries the map entries
+     */
+    public static <T extends AssignmentTestable> AssignmentTestable.TestResult testMapAssignment(ColumnSpecification receiver,
+                                                                                                 List<Pair<T, T>> entries)
+    {
+        ColumnSpecification keySpec = keySpecOf(receiver);
+        ColumnSpecification valueSpec = valueSpecOf(receiver);
+
+        // It's an exact match if all are exact match, but is not assignable as soon as any is non assignable.
+        AssignmentTestable.TestResult res = AssignmentTestable.TestResult.EXACT_MATCH;
+        for (Pair<T, T> entry : entries)
+        {
+            AssignmentTestable.TestResult t1 = entry.left.testAssignment(receiver.ksName, keySpec);
+            AssignmentTestable.TestResult t2 = entry.right.testAssignment(receiver.ksName, valueSpec);
+            if (t1 == AssignmentTestable.TestResult.NOT_ASSIGNABLE || t2 == AssignmentTestable.TestResult.NOT_ASSIGNABLE)
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+            if (t1 != AssignmentTestable.TestResult.EXACT_MATCH || t2 != AssignmentTestable.TestResult.EXACT_MATCH)
+                res = AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+        }
+        return res;
+    }
+
+    /**
+     * Create a <code>String</code> representation of the list containing the specified elements.
+     *
+     * @param elements the list elements
+     * @return a <code>String</code> representation of the list
+     */
+    public static <T> String mapToString(List<Pair<T, T>> entries)
+    {
+        return mapToString(entries, Object::toString);
+    }
+
+    /**
+     * Create a <code>String</code> representation of the map from the specified items associated to
+     * the map entries.
+     *
+     * @param items items associated to the map entries
+     * @param mapper the mapper used to map the items to the <code>String</code> representation of the map entries
+     * @return a <code>String</code> representation of the map
+     */
+    public static <T> String mapToString(List<Pair<T, T>> items,
+                                         java.util.function.Function<T, String> mapper)
+    {
+        return items.stream()
+                .map(p -> String.format("%s: %s", mapper.apply(p.left), mapper.apply(p.right)))
+                .collect(Collectors.joining(", ", "{", "}"));
+    }
+
+    /**
+     * Returns the exact MapType from the entries if it can be known.
+     *
+     * @param entries the entries
+     * @param mapper the mapper used to retrieve the key and value types from the entries
+     * @return the exact MapType from the entries if it can be known or <code>null</code>
+     */
+    public static <T> AbstractType<?> getExactMapTypeIfKnown(List<Pair<T, T>> entries,
+                                                             java.util.function.Function<T, AbstractType<?>> mapper)
+    {
+        AbstractType<?> keyType = null;
+        AbstractType<?> valueType = null;
+        for (Pair<T, T> entry : entries)
+        {
+            if (keyType == null)
+                keyType = mapper.apply(entry.left);
+            if (valueType == null)
+                valueType = mapper.apply(entry.right);
+            if (keyType != null && valueType != null)
+                return MapType.getInstance(keyType, valueType, false);
+        }
+        return null;
     }
 
     public static class Literal extends Term.Raw
@@ -82,7 +159,7 @@ public abstract class Maps
 
                 values.put(k, v);
             }
-            DelayedValue value = new DelayedValue(((MapType)receiver.type).getKeysType(), values);
+            DelayedValue value = new DelayedValue(((MapType<?, ?>)receiver.type).getKeysType(), values);
             return allTerminal ? value.bind(QueryOptions.DEFAULT) : value;
         }
 
@@ -104,51 +181,18 @@ public abstract class Maps
 
         public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            if (!(receiver.type instanceof MapType))
-                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
-
-            // If there is no elements, we can't say it's an exact match (an empty map if fundamentally polymorphic).
-            if (entries.isEmpty())
-                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
-
-            ColumnSpecification keySpec = Maps.keySpecOf(receiver);
-            ColumnSpecification valueSpec = Maps.valueSpecOf(receiver);
-            // It's an exact match if all are exact match, but is not assignable as soon as any is non assignable.
-            AssignmentTestable.TestResult res = AssignmentTestable.TestResult.EXACT_MATCH;
-            for (Pair<Term.Raw, Term.Raw> entry : entries)
-            {
-                AssignmentTestable.TestResult t1 = entry.left.testAssignment(keyspace, keySpec);
-                AssignmentTestable.TestResult t2 = entry.right.testAssignment(keyspace, valueSpec);
-                if (t1 == AssignmentTestable.TestResult.NOT_ASSIGNABLE || t2 == AssignmentTestable.TestResult.NOT_ASSIGNABLE)
-                    return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
-                if (t1 != AssignmentTestable.TestResult.EXACT_MATCH || t2 != AssignmentTestable.TestResult.EXACT_MATCH)
-                    res = AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
-            }
-            return res;
+            return testMapAssignment(receiver, entries);
         }
 
         @Override
         public AbstractType<?> getExactTypeIfKnown(String keyspace)
         {
-            AbstractType<?> keyType = null;
-            AbstractType<?> valueType = null;
-            for (Pair<Term.Raw, Term.Raw> entry : entries)
-            {
-                if (keyType == null)
-                    keyType = entry.left.getExactTypeIfKnown(keyspace);
-                if (valueType == null)
-                    valueType = entry.right.getExactTypeIfKnown(keyspace);
-                if (keyType != null && valueType != null)
-                    return MapType.getInstance(keyType, valueType, false);
-            }
-            return null;
+            return getExactMapTypeIfKnown(entries, p -> p.getExactTypeIfKnown(keyspace));
         }
 
         public String getText()
         {
-            return entries.stream()
-                    .map(entry -> String.format("%s: %s", entry.left.getText(), entry.right.getText()))
-                    .collect(Collectors.joining(", ", "{", "}"));
+            return mapToString(entries, Term.Raw::getText);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/Sets.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java
index d17a771..e79bda7 100644
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@ -22,6 +22,7 @@ import static org.apache.cassandra.cql3.Constants.UNSET_VALUE;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
@@ -43,7 +44,73 @@ public abstract class Sets
 
     public static ColumnSpecification valueSpecOf(ColumnSpecification column)
     {
-        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((SetType)column.type).getElementsType());
+        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((SetType<?>)column.type).getElementsType());
+    }
+
+    /**
+     * Tests that the set with the specified elements can be assigned to the specified column.
+     *
+     * @param receiver the receiving column
+     * @param elements the set elements
+     */
+    public static AssignmentTestable.TestResult testSetAssignment(ColumnSpecification receiver,
+                                                                  List<? extends AssignmentTestable> elements)
+    {
+        if (!(receiver.type instanceof SetType))
+        {
+            // We've parsed empty maps as a set literal to break the ambiguity so handle that case now
+            if (receiver.type instanceof MapType && elements.isEmpty())
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+            return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+        }
+
+        // If there is no elements, we can't say it's an exact match (an empty set if fundamentally polymorphic).
+        if (elements.isEmpty())
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+        ColumnSpecification valueSpec = valueSpecOf(receiver);
+        return AssignmentTestable.TestResult.testAll(receiver.ksName, valueSpec, elements);
+    }
+
+    /**
+     * Create a <code>String</code> representation of the set containing the specified elements.
+     *
+     * @param elements the set elements
+     * @return a <code>String</code> representation of the set
+     */
+    public static String setToString(List<?> elements)
+    {
+        return setToString(elements, Object::toString);
+    }
+
+    /**
+     * Create a <code>String</code> representation of the set from the specified items associated to
+     * the set elements.
+     *
+     * @param items items associated to the set elements
+     * @param mapper the mapper used to map the items to the <code>String</code> representation of the set elements
+     * @return a <code>String</code> representation of the set
+     */
+    public static <T> String setToString(Iterable<T> items, java.util.function.Function<T, String> mapper)
+    {
+        return StreamSupport.stream(items.spliterator(), false)
+                            .map(e -> mapper.apply(e))
+                            .collect(Collectors.joining(", ", "{", "}"));
+    }
+
+    /**
+     * Returns the exact SetType from the items if it can be known.
+     *
+     * @param items the items mapped to the set elements
+     * @param mapper the mapper used to retrieve the element types from the items
+     * @return the exact SetType from the items if it can be known or <code>null</code>
+     */
+    public static <T> AbstractType<?> getExactSetTypeIfKnown(List<T> items,
+                                                             java.util.function.Function<T, AbstractType<?>> mapper)
+    {
+        Optional<AbstractType<?>> type = items.stream().map(mapper).filter(Objects::nonNull).findFirst();
+        return type.isPresent() ? SetType.getInstance(type.get(), false) : null;
     }
 
     public static class Literal extends Term.Raw
@@ -105,38 +172,18 @@ public abstract class Sets
 
         public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            if (!(receiver.type instanceof SetType))
-            {
-                // We've parsed empty maps as a set literal to break the ambiguity so handle that case now
-                if (receiver.type instanceof MapType && elements.isEmpty())
-                    return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
-
-                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
-            }
-
-            // If there is no elements, we can't say it's an exact match (an empty set if fundamentally polymorphic).
-            if (elements.isEmpty())
-                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
-
-            ColumnSpecification valueSpec = Sets.valueSpecOf(receiver);
-            return AssignmentTestable.TestResult.testAll(keyspace, valueSpec, elements);
+            return testSetAssignment(receiver, elements);
         }
 
         @Override
         public AbstractType<?> getExactTypeIfKnown(String keyspace)
         {
-            for (Term.Raw term : elements)
-            {
-                AbstractType<?> type = term.getExactTypeIfKnown(keyspace);
-                if (type != null)
-                    return SetType.getInstance(type, false);
-            }
-            return null;
+            return getExactSetTypeIfKnown(elements, p -> p.getExactTypeIfKnown(keyspace));
         }
 
         public String getText()
         {
-            return elements.stream().map(Term.Raw::getText).collect(Collectors.joining(", ", "{", "}"));
+            return setToString(elements, Term.Raw::getText);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index 32a31fd..bae756a 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,6 +34,8 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * Static helper methods and classes for tuples.
  */
@@ -65,7 +68,12 @@ public class Tuples
 
         public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
-            validateAssignableTo(keyspace, receiver);
+            // The parser cannot differentiate between a tuple with one element and a term between parenthesis.
+            // By consequence, we need to wait until we know the target type to determine which one it is.
+            if (elements.size() == 1 && !(receiver.type instanceof TupleType))
+                return elements.get(0).prepare(keyspace, receiver);
+
+            validateTupleAssignableTo(receiver, elements);
 
             List<Term> values = new ArrayList<>(elements.size());
             boolean allTerminal = true;
@@ -102,38 +110,14 @@ public class Tuples
             return allTerminal ? value.bind(QueryOptions.DEFAULT) : value;
         }
 
-        private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
-        {
-            if (!(receiver.type instanceof TupleType))
-                throw new InvalidRequestException(String.format("Invalid tuple type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type()));
-
-            TupleType tt = (TupleType)receiver.type;
-            for (int i = 0; i < elements.size(); i++)
-            {
-                if (i >= tt.size())
-                {
-                    throw new InvalidRequestException(String.format("Invalid tuple literal for %s: too many elements. Type %s expects %d but got %d",
-                            receiver.name, tt.asCQL3Type(), tt.size(), elements.size()));
-                }
-
-                Term.Raw value = elements.get(i);
-                ColumnSpecification spec = componentSpecOf(receiver, i);
-                if (!value.testAssignment(keyspace, spec).isAssignable())
-                    throw new InvalidRequestException(String.format("Invalid tuple literal for %s: component %d is not of type %s", receiver.name, i, spec.type.asCQL3Type()));
-            }
-        }
-
         public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            try
-            {
-                validateAssignableTo(keyspace, receiver);
-                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
-            }
-            catch (InvalidRequestException e)
-            {
-                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
-            }
+            // The parser cannot differentiate between a tuple with one element and a term between parenthesis.
+            // By consequence, we need to wait until we know the target type to determine which one it is.
+            if (elements.size() == 1 && !(receiver.type instanceof TupleType))
+                return elements.get(0).testAssignment(keyspace, receiver);
+
+            return testTupleAssignment(receiver, elements);
         }
 
         @Override
@@ -152,7 +136,7 @@ public class Tuples
 
         public String getText()
         {
-            return elements.stream().map(Term.Raw::getText).collect(Collectors.joining(", ", "(", ")"));
+            return tupleToString(elements, Term.Raw::getText);
         }
     }
 
@@ -436,17 +420,100 @@ public class Tuples
         }
     }
 
-    public static String tupleToString(List<?> items)
+    /**
+     * Create a <code>String</code> representation of the tuple containing the specified elements.
+     *
+     * @param elements the tuple elements
+     * @return a <code>String</code> representation of the tuple
+     */
+    public static String tupleToString(List<?> elements)
+    {
+        return tupleToString(elements, Object::toString);
+    }
+
+    /**
+     * Create a <code>String</code> representation of the tuple from the specified items associated to
+     * the tuples elements.
+     *
+     * @param items items associated to the tuple elements
+     * @param mapper the mapper used to map the items to the <code>String</code> representation of the tuple elements
+     * @return a <code>String</code> representation of the tuple
+     */
+    public static <T> String tupleToString(Iterable<T> items, java.util.function.Function<T, String> mapper)
+    {
+        return StreamSupport.stream(items.spliterator(), false)
+                            .map(e -> mapper.apply(e))
+                            .collect(Collectors.joining(", ", "(", ")"));
+    }
+
+    /**
+     * Returns the exact TupleType from the items if it can be known.
+     *
+     * @param items the items mapped to the tuple elements
+     * @param mapper the mapper used to retrieve the element types from the  items
+     * @return the exact TupleType from the items if it can be known or <code>null</code>
+     */
+    public static <T> AbstractType<?> getExactTupleTypeIfKnown(List<T> items,
+                                                               java.util.function.Function<T, AbstractType<?>> mapper)
+    {
+        List<AbstractType<?>> types = new ArrayList<>(items.size());
+        for (T item : items)
+        {
+            AbstractType<?> type = mapper.apply(item);
+            if (type == null)
+                return null;
+            types.add(type);
+        }
+        return new TupleType(types);
+    }
+
+    /**
+     * Checks if the tuple with the specified elements can be assigned to the specified column.
+     *
+     * @param receiver the receiving column
+     * @param elements the tuple elements
+     * @throws InvalidRequestException if the tuple cannot be assigned to the specified column.
+     */
+    public static void validateTupleAssignableTo(ColumnSpecification receiver,
+                                                 List<? extends AssignmentTestable> elements)
     {
+        if (!(receiver.type instanceof TupleType))
+            throw invalidRequest("Invalid tuple type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type());
 
-        StringBuilder sb = new StringBuilder("(");
-        for (int i = 0; i < items.size(); i++)
+        TupleType tt = (TupleType)receiver.type;
+        for (int i = 0; i < elements.size(); i++)
+        {
+            if (i >= tt.size())
+            {
+                throw invalidRequest("Invalid tuple literal for %s: too many elements. Type %s expects %d but got %d",
+                                     receiver.name, tt.asCQL3Type(), tt.size(), elements.size());
+            }
+
+            AssignmentTestable value = elements.get(i);
+            ColumnSpecification spec = componentSpecOf(receiver, i);
+            if (!value.testAssignment(receiver.ksName, spec).isAssignable())
+                throw invalidRequest("Invalid tuple literal for %s: component %d is not of type %s",
+                                     receiver.name, i, spec.type.asCQL3Type());
+        }
+    }
+
+    /**
+     * Tests that the tuple with the specified elements can be assigned to the specified column.
+     *
+     * @param receiver the receiving column
+     * @param elements the tuple elements
+     */
+    public static AssignmentTestable.TestResult testTupleAssignment(ColumnSpecification receiver,
+                                                                    List<? extends AssignmentTestable> elements)
+    {
+        try
+        {
+            validateTupleAssignableTo(receiver, elements);
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+        }
+        catch (InvalidRequestException e)
         {
-            sb.append(items.get(i));
-            if (i < items.size() - 1)
-                sb.append(", ");
+            return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
         }
-        sb.append(')');
-        return sb.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/UserTypes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UserTypes.java b/src/java/org/apache/cassandra/cql3/UserTypes.java
index e867179..ae62177 100644
--- a/src/java/org/apache/cassandra/cql3/UserTypes.java
+++ b/src/java/org/apache/cassandra/cql3/UserTypes.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.stream.Collectors;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
@@ -47,6 +48,78 @@ public abstract class UserTypes
                                        ut.fieldType(field));
     }
 
+    public static <T extends AssignmentTestable> void validateUserTypeAssignableTo(ColumnSpecification receiver,
+                                                                                   Map<FieldIdentifier, T> entries)
+    {
+        if (!receiver.type.isUDT())
+            throw new InvalidRequestException(String.format("Invalid user type literal for %s of type %s", receiver, receiver.type.asCQL3Type()));
+
+        UserType ut = (UserType) receiver.type;
+        for (int i = 0; i < ut.size(); i++)
+        {
+            FieldIdentifier field = ut.fieldName(i);
+            T value = entries.get(field);
+            if (value == null)
+                continue;
+
+            ColumnSpecification fieldSpec = fieldSpecOf(receiver, i);
+            if (!value.testAssignment(receiver.ksName, fieldSpec).isAssignable())
+            {
+                throw new InvalidRequestException(String.format("Invalid user type literal for %s: field %s is not of type %s",
+                        receiver, field, fieldSpec.type.asCQL3Type()));
+            }
+        }
+    }
+
+    /**
+     * Tests that the map with the specified entries can be assigned to the specified column.
+     *
+     * @param receiver the receiving column
+     * @param entries the map entries
+     */
+    public static <T extends AssignmentTestable> AssignmentTestable.TestResult testUserTypeAssignment(ColumnSpecification receiver,
+                                                                                                      Map<FieldIdentifier, T> entries)
+    {
+        try
+        {
+            validateUserTypeAssignableTo(receiver, entries);
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+        }
+        catch (InvalidRequestException e)
+        {
+            return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+        }
+    }
+
+    /**
+     * Create a {@code String} representation of the user type from the specified items associated to
+     * the user type entries.
+     *
+     * @param items items associated to the user type entries
+     * @param mapper the mapper used to user type the items to the {@code String} representation of the map entries
+     * @return a {@code String} representation of the user type
+     */
+    public static <T> String userTypeToString(Map<FieldIdentifier, T> items)
+    {
+        return userTypeToString(items, Object::toString);
+    }
+
+    /**
+     * Create a {@code String} representation of the user type from the specified items associated to
+     * the user type entries.
+     *
+     * @param items items associated to the user type entries
+     * @return a {@code String} representation of the user type
+     */
+    public static <T> String userTypeToString(Map<FieldIdentifier, T> items,
+                                              java.util.function.Function<T, String> mapper)
+    {
+        return items.entrySet()
+                    .stream()
+                    .map(p -> String.format("%s: %s", p.getKey(), mapper.apply(p.getValue())))
+                    .collect(Collectors.joining(", ", "{", "}"));
+    }
+
     public static class Literal extends Term.Raw
     {
         public final Map<FieldIdentifier, Term.Raw> entries;
@@ -95,37 +168,12 @@ public abstract class UserTypes
 
         private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
-            if (!receiver.type.isUDT())
-                throw new InvalidRequestException(String.format("Invalid user type literal for %s of type %s", receiver, receiver.type.asCQL3Type()));
-
-            UserType ut = (UserType)receiver.type;
-            for (int i = 0; i < ut.size(); i++)
-            {
-                FieldIdentifier field = ut.fieldName(i);
-                Term.Raw value = entries.get(field);
-                if (value == null)
-                    continue;
-
-                ColumnSpecification fieldSpec = fieldSpecOf(receiver, i);
-                if (!value.testAssignment(keyspace, fieldSpec).isAssignable())
-                {
-                    throw new InvalidRequestException(String.format("Invalid user type literal for %s: field %s is not of type %s",
-                            receiver, field, fieldSpec.type.asCQL3Type()));
-                }
-            }
+            validateUserTypeAssignableTo(receiver, entries);
         }
 
         public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            try
-            {
-                validateAssignableTo(keyspace, receiver);
-                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
-            }
-            catch (InvalidRequestException e)
-            {
-                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
-            }
+            return testUserTypeAssignment(receiver, entries);
         }
 
         public AbstractType<?> getExactTypeIfKnown(String keyspace)
@@ -135,18 +183,7 @@ public abstract class UserTypes
 
         public String getText()
         {
-            StringBuilder sb = new StringBuilder();
-            sb.append("{");
-            Iterator<Map.Entry<FieldIdentifier, Term.Raw>> iter = entries.entrySet().iterator();
-            while (iter.hasNext())
-            {
-                Map.Entry<FieldIdentifier, Term.Raw> entry = iter.next();
-                sb.append(entry.getKey()).append(": ").append(entry.getValue().getText());
-                if (iter.hasNext())
-                    sb.append(", ");
-            }
-            sb.append("}");
-            return sb.toString();
+            return userTypeToString(entries, Term.Raw::getText);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
index bcc912f..b8f5652 100644
--- a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.cql3.functions;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.stream.Collectors;
 
@@ -30,6 +32,8 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 public class FunctionCall extends Term.NonTerminal
 {
     private final ScalarFunction fun;
@@ -130,26 +134,45 @@ public class FunctionCall extends Term.NonTerminal
             this.terms = terms;
         }
 
+        public static Raw newOperation(char operator, Term.Raw left, Term.Raw right)
+        {
+            FunctionName name = OperationFcts.getFunctionNameFromOperator(operator);
+            return new Raw(name, Arrays.asList(left, right));
+        }
+
+        public static Raw newNegation(Term.Raw raw)
+        {
+            FunctionName name = FunctionName.nativeFunction(OperationFcts.NEGATION_FUNCTION_NAME);
+            return new Raw(name, Collections.singletonList(raw));
+        }
+
         public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
             Function fun = FunctionResolver.get(keyspace, name, terms, receiver.ksName, receiver.cfName, receiver.type);
             if (fun == null)
-                throw new InvalidRequestException(String.format("Unknown function %s called", name));
+                throw invalidRequest("Unknown function %s called", name);
             if (fun.isAggregate())
-                throw new InvalidRequestException("Aggregation function are not supported in the where clause");
+                throw invalidRequest("Aggregation function are not supported in the where clause");
 
             ScalarFunction scalarFun = (ScalarFunction) fun;
 
             // Functions.get() will complain if no function "name" type check with the provided arguments.
             // We still have to validate that the return type matches however
             if (!scalarFun.testAssignment(keyspace, receiver).isAssignable())
-                throw new InvalidRequestException(String.format("Type error: cannot assign result of function %s (type %s) to %s (type %s)",
-                                                                scalarFun.name(), scalarFun.returnType().asCQL3Type(),
-                                                                receiver.name, receiver.type.asCQL3Type()));
+            {
+                if (OperationFcts.isOperation(name))
+                    throw invalidRequest("Type error: cannot assign result of operation %s (type %s) to %s (type %s)",
+                                         OperationFcts.getOperator(scalarFun.name()), scalarFun.returnType().asCQL3Type(),
+                                         receiver.name, receiver.type.asCQL3Type());
+
+                throw invalidRequest("Type error: cannot assign result of function %s (type %s) to %s (type %s)",
+                                     scalarFun.name(), scalarFun.returnType().asCQL3Type(),
+                                     receiver.name, receiver.type.asCQL3Type());
+            }
 
             if (fun.argTypes().size() != terms.size())
-                throw new InvalidRequestException(String.format("Incorrect number of arguments specified for function %s (expected %d, found %d)",
-                                                                fun, fun.argTypes().size(), terms.size()));
+                throw invalidRequest("Incorrect number of arguments specified for function %s (expected %d, found %d)",
+                                     fun, fun.argTypes().size(), terms.size());
 
             List<Term> parameters = new ArrayList<>(terms.size());
             for (int i = 0; i < terms.size(); i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/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 9e0b706..7234d1f 100644
--- a/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
@@ -22,11 +22,15 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.AbstractMarker;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 import static java.util.stream.Collectors.joining;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 
 public final class FunctionResolver
 {
@@ -112,33 +116,83 @@ public final class FunctionResolver
         List<Function> compatibles = null;
         for (Function toTest : candidates)
         {
-            AssignmentTestable.TestResult r = matchAguments(keyspace, toTest, providedArgs, receiverKs, receiverCf);
-            switch (r)
+            if (matchReturnType(toTest, receiverType))
             {
-                case EXACT_MATCH:
-                    // We always favor exact matches
-                    return toTest;
-                case WEAKLY_ASSIGNABLE:
-                    if (compatibles == null)
-                        compatibles = new ArrayList<>();
-                    compatibles.add(toTest);
-                    break;
+                AssignmentTestable.TestResult r = matchAguments(keyspace, toTest, providedArgs, receiverKs, receiverCf);
+                switch (r)
+                {
+                    case EXACT_MATCH:
+                        // We always favor exact matches
+                        return toTest;
+                    case WEAKLY_ASSIGNABLE:
+                        if (compatibles == null)
+                            compatibles = new ArrayList<>();
+                        compatibles.add(toTest);
+                        break;
+                }
             }
         }
 
         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 (OperationFcts.isOperation(name))
+                throw invalidRequest("the '%s' operation is not supported between %s and %s",
+                                     OperationFcts.getOperator(name), providedArgs.get(0), providedArgs.get(1));
+
+            throw invalidRequest("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",
-                        name, format(compatibles)));
+        {
+            if (OperationFcts.isOperation(name))
+            {
+                if (receiverType != null && !containsMarkers(providedArgs))
+                {
+                    for (Function toTest : compatibles)
+                    {
+                        List<AbstractType<?>> argTypes = toTest.argTypes();
+                        if (receiverType.equals(argTypes.get(0)) && receiverType.equals(argTypes.get(1)))
+                            return toTest;
+                    }
+                }
+                throw invalidRequest("Ambiguous '%s' operation: use type casts to disambiguate",
+                                     OperationFcts.getOperator(name), providedArgs.get(0), providedArgs.get(1));
+            }
+
+            if (OperationFcts.isNegation(name))
+                throw invalidRequest("Ambiguous negation: use type casts to disambiguate");
 
+            throw invalidRequest("Ambiguous call to function %s (can be matched by following signatures: %s): use type casts to disambiguate",
+                                 name, format(compatibles));
+        }
         return compatibles.get(0);
     }
 
+    /**
+     * Checks if at least one of the specified arguments is a marker.
+     *
+     * @param args the arguments to check
+     * @return {@code true} if if at least one of the specified arguments is a marker, {@code false} otherwise
+     */
+    private static boolean containsMarkers(List<? extends AssignmentTestable> args)
+    {
+        return args.stream().anyMatch(AbstractMarker.Raw.class::isInstance);
+    }
+
+    /**
+     * Checks that the return type of the specified function can be assigned to the specified receiver.
+     *
+     * @param fun the function to check
+     * @param receiverType the receiver type
+     * @return {@code true} if the return type of the specified function can be assigned to the specified receiver,
+     * {@code false} otherwise.
+     */
+    private static boolean matchReturnType(Function fun, AbstractType<?> receiverType)
+    {
+        return receiverType == null || fun.returnType().testAssignment(receiverType).isAssignable();
+    }
+
     // This method and matchArguments are somewhat duplicate, but this method allows us to provide more precise errors in the common
     // case where there is no override for a given function. This is thus probably worth the minor code duplication.
     private static void validateTypes(String keyspace,
@@ -146,10 +200,10 @@ public final class FunctionResolver
                                       List<? extends AssignmentTestable> providedArgs,
                                       String receiverKs,
                                       String receiverCf)
-    throws InvalidRequestException
     {
         if (providedArgs.size() != fun.argTypes().size())
-            throw new InvalidRequestException(String.format("Invalid number of arguments in call to function %s: %d required but %d provided", fun.name(), fun.argTypes().size(), providedArgs.size()));
+            throw invalidRequest("Invalid number of arguments in call to function %s: %d required but %d provided",
+                                 fun.name(), fun.argTypes().size(), providedArgs.size());
 
         for (int i = 0; i < providedArgs.size(); i++)
         {
@@ -162,7 +216,8 @@ public final class FunctionResolver
 
             ColumnSpecification expected = makeArgSpec(receiverKs, receiverCf, fun, i);
             if (!provided.testAssignment(keyspace, expected).isAssignable())
-                throw new InvalidRequestException(String.format("Type error: %s cannot be passed as argument %d of function %s of type %s", provided, i, fun.name(), expected.type.asCQL3Type()));
+                throw invalidRequest("Type error: %s cannot be passed as argument %d of function %s of type %s",
+                                     provided, i, fun.name(), expected.type.asCQL3Type());
         }
     }