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:47 UTC

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

Repository: cassandra
Updated Branches:
  refs/heads/trunk 58cf4c907 -> f782f148b


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
index ece2d1d..975eb8e 100644
--- a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
+++ b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static java.util.Arrays.asList;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -102,6 +103,14 @@ public class SelectionColumnMappingTest extends CQLTester
         testMixedColumnTypes();
         testMultipleUnaliasedSelectionOfSameColumn();
         testUserDefinedAggregate();
+        testListLitteral();
+        testEmptyListLitteral();
+        testSetLitteral();
+        testEmptySetLitteral();
+        testMapLitteral();
+        testEmptyMapLitteral();
+        testUDTLitteral();
+        testTupleLitteral();
     }
 
     @Test
@@ -407,6 +416,91 @@ public class SelectionColumnMappingTest extends CQLTester
         verify(expected, "SELECT v1, v1 FROM %s");
     }
 
+    private void testListLitteral() throws Throwable
+    {
+        ColumnSpecification listSpec = columnSpecification("[k, v1]", ListType.getInstance(Int32Type.instance, false));
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(listSpec, asList(columnDefinition("k"),
+                                                                                             columnDefinition("v1")));
+
+        verify(expected, "SELECT [k, v1] FROM %s");
+    }
+
+    private void testEmptyListLitteral() throws Throwable
+    {
+        ColumnSpecification listSpec = columnSpecification("(list<int>)[]", ListType.getInstance(Int32Type.instance, false));
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(listSpec, (ColumnDefinition) null);
+
+        verify(expected, "SELECT (list<int>)[] FROM %s");
+    }
+
+    private void testSetLitteral() throws Throwable
+    {
+        ColumnSpecification setSpec = columnSpecification("{k, v1}", SetType.getInstance(Int32Type.instance, false));
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(setSpec, asList(columnDefinition("k"),
+                                                                                             columnDefinition("v1")));
+
+        verify(expected, "SELECT {k, v1} FROM %s");
+    }
+
+    private void testEmptySetLitteral() throws Throwable
+    {
+        ColumnSpecification setSpec = columnSpecification("(set<int>){}", SetType.getInstance(Int32Type.instance, false));
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(setSpec, (ColumnDefinition) null);
+
+        verify(expected, "SELECT (set<int>){} FROM %s");
+    }
+
+    private void testMapLitteral() throws Throwable
+    {
+        ColumnSpecification mapSpec = columnSpecification("(map<text, int>){'min': system.min(v1), 'max': system.max(v1)}", MapType.getInstance(UTF8Type.instance, Int32Type.instance, false));
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(mapSpec, asList(columnDefinition("v1")));
+
+        verify(expected, "SELECT (map<text, int>){'min': min(v1), 'max': max(v1)} FROM %s");
+    }
+
+    private void testEmptyMapLitteral() throws Throwable
+    {
+        ColumnSpecification mapSpec = columnSpecification("(map<text, int>){}", MapType.getInstance(UTF8Type.instance, Int32Type.instance, false));
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(mapSpec, (ColumnDefinition) null);
+
+        verify(expected, "SELECT (map<text, int>){} FROM %s");
+    }
+
+    private void testUDTLitteral() throws Throwable
+    {
+        UserType type = new UserType(KEYSPACE, ByteBufferUtil.bytes(typeName),
+                                      asList(FieldIdentifier.forUnquoted("f1"),
+                                             FieldIdentifier.forUnquoted("f2")),
+                                      asList(Int32Type.instance,
+                                             UTF8Type.instance),
+                                      false);
+
+        ColumnSpecification spec = columnSpecification("(" + KEYSPACE + "." + typeName + "){f1: v1, f2: v2}", type);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(spec, asList(columnDefinition("v1"),
+                                                                                         columnDefinition("v2")));
+
+        verify(expected, "SELECT ("+ typeName + "){f1: v1, f2: v2} FROM %s");
+    }
+
+    private void testTupleLitteral() throws Throwable
+    {
+        TupleType type = new TupleType(asList(Int32Type.instance, UTF8Type.instance));
+
+        ColumnSpecification setSpec = columnSpecification("(tuple<int, text>)(v1, v2)", type);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(setSpec, asList(columnDefinition("v1"),
+                                                                                            columnDefinition("v2")));
+
+        verify(expected, "SELECT (tuple<int, text>)(v1, v2) FROM %s");
+    }
+
     private void testMixedColumnTypes() throws Throwable
     {
         ColumnSpecification kSpec = columnSpecification("k_alias", Int32Type.instance);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java b/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
index a07f8f9..fb46809 100644
--- a/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
@@ -43,20 +43,26 @@ public class TermSelectionTest extends CQLTester
     @Test
     public void testSelectLiteral() throws Throwable
     {
+        long timestampInMicros = System.currentTimeMillis() * 1000;
         createTable("CREATE TABLE %s (pk int, ck int, t text, PRIMARY KEY (pk, ck) )");
-        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 1, 'one')");
-        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 2, 'two')");
-        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 3, 'three')");
+        execute("INSERT INTO %s (pk, ck, t) VALUES (?, ?, ?) USING TIMESTAMP ?", 1, 1, "one", timestampInMicros);
+        execute("INSERT INTO %s (pk, ck, t) VALUES (?, ?, ?) USING TIMESTAMP ?", 1, 2, "two", timestampInMicros);
+        execute("INSERT INTO %s (pk, ck, t) VALUES (?, ?, ?) USING TIMESTAMP ?", 1, 3, "three", timestampInMicros);
 
         assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, 'a const' FROM %s");
         assertConstantResult(execute("SELECT ck, t, (text)'a const' FROM %s"), "a const");
 
         assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, 42 FROM %s");
-        assertConstantResult(execute("SELECT ck, t, (int)42 FROM %s"), 42);
+        assertConstantResult(execute("SELECT ck, t, (smallint)42 FROM %s"), (short) 42);
 
         assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, (1, 'foo') FROM %s");
         assertConstantResult(execute("SELECT ck, t, (tuple<int, text>)(1, 'foo') FROM %s"), tuple(1, "foo"));
 
+        assertInvalidMessage("Cannot infer type for term ((1)) in selection clause", "SELECT ck, t, ((1)) FROM %s");
+        // We cannot differentiate a tuple containing a tuple from a tuple between parentheses.
+        assertInvalidMessage("Cannot infer type for term ((tuple<int>)(1))", "SELECT ck, t, ((tuple<int>)(1)) FROM %s");
+        assertConstantResult(execute("SELECT ck, t, (tuple<tuple<int>>)((1)) FROM %s"), tuple(tuple(1)));
+
         assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, [1, 2, 3] FROM %s");
         assertConstantResult(execute("SELECT ck, t, (list<int>)[1, 2, 3] FROM %s"), list(1, 2, 3));
 
@@ -66,11 +72,343 @@ public class TermSelectionTest extends CQLTester
         assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, {1: 'foo', 2: 'bar', 3: 'baz'} FROM %s");
         assertConstantResult(execute("SELECT ck, t, (map<int, text>){1: 'foo', 2: 'bar', 3: 'baz'} FROM %s"), map(1, "foo", 2, "bar", 3, "baz"));
 
+        assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, {} FROM %s");
+        assertConstantResult(execute("SELECT ck, t, (map<int, text>){} FROM %s"), map());
+        assertConstantResult(execute("SELECT ck, t, (set<int>){} FROM %s"), set());
+
         assertColumnNames(execute("SELECT ck, t, (int)42, (int)43 FROM %s"), "ck", "t", "(int)42", "(int)43");
         assertRows(execute("SELECT ck, t, (int) 42, (int) 43 FROM %s"),
                    row(1, "one", 42, 43),
                    row(2, "two", 42, 43),
                    row(3, "three", 42, 43));
+
+        assertRows(execute("SELECT min(ck), max(ck), [min(ck), max(ck)] FROM %s"), row(1, 3, list(1, 3)));
+        assertRows(execute("SELECT [min(ck), max(ck)] FROM %s"), row(list(1, 3)));
+        assertRows(execute("SELECT {min(ck), max(ck)} FROM %s"), row(set(1, 3)));
+
+        // We need to use a cast to differentiate between a map and an UDT
+        assertInvalidMessage("Cannot infer type for term {'min': system.min(ck), 'max': system.max(ck)}",
+                             "SELECT {'min' : min(ck), 'max' : max(ck)} FROM %s");
+        assertRows(execute("SELECT (map<text, int>){'min' : min(ck), 'max' : max(ck)} FROM %s"), row(map("min", 1, "max", 3)));
+
+        assertRows(execute("SELECT [1, min(ck), max(ck)] FROM %s"), row(list(1, 1, 3)));
+        assertRows(execute("SELECT {1, min(ck), max(ck)} FROM %s"), row(set(1, 1, 3)));
+        assertRows(execute("SELECT (map<text, int>) {'litteral' : 1, 'min' : min(ck), 'max' : max(ck)} FROM %s"), row(map("litteral", 1, "min", 1, "max", 3)));
+
+        // Test List nested within Lists
+        assertRows(execute("SELECT [[], [min(ck), max(ck)]] FROM %s"),
+                   row(list(list(), list(1, 3))));
+        assertRows(execute("SELECT [[], [CAST(pk AS BIGINT), CAST(ck AS BIGINT), WRITETIME(t)]] FROM %s"),
+                   row(list(list(), list(1L, 1L, timestampInMicros))),
+                   row(list(list(), list(1L, 2L, timestampInMicros))),
+                   row(list(list(), list(1L, 3L, timestampInMicros))));
+        assertRows(execute("SELECT [[min(ck)], [max(ck)]] FROM %s"),
+                   row(list(list(1), list(3))));
+        assertRows(execute("SELECT [[min(ck)], ([max(ck)])] FROM %s"),
+                   row(list(list(1), list(3))));
+        assertRows(execute("SELECT [[pk], [ck]] FROM %s"),
+                   row(list(list(1), list(1))),
+                   row(list(list(1), list(2))),
+                   row(list(list(1), list(3))));
+        assertRows(execute("SELECT [[pk], [ck]] FROM %s WHERE pk = 1 ORDER BY ck DESC"),
+                   row(list(list(1), list(3))),
+                   row(list(list(1), list(2))),
+                   row(list(list(1), list(1))));
+
+        // Test Sets nested within Lists
+        assertRows(execute("SELECT [{}, {min(ck), max(ck)}] FROM %s"),
+                   row(list(set(), set(1, 3))));
+        assertRows(execute("SELECT [{}, {CAST(pk AS BIGINT), CAST(ck AS BIGINT), WRITETIME(t)}] FROM %s"),
+                   row(list(set(), set(1L, 1L, timestampInMicros))),
+                   row(list(set(), set(1L, 2L, timestampInMicros))),
+                   row(list(set(), set(1L, 3L, timestampInMicros))));
+        assertRows(execute("SELECT [{min(ck)}, {max(ck)}] FROM %s"),
+                   row(list(set(1), set(3))));
+        assertRows(execute("SELECT [{min(ck)}, ({max(ck)})] FROM %s"),
+                   row(list(set(1), set(3))));
+        assertRows(execute("SELECT [{pk}, {ck}] FROM %s"),
+                   row(list(set(1), set(1))),
+                   row(list(set(1), set(2))),
+                   row(list(set(1), set(3))));
+        assertRows(execute("SELECT [{pk}, {ck}] FROM %s WHERE pk = 1 ORDER BY ck DESC"),
+                   row(list(set(1), set(3))),
+                   row(list(set(1), set(2))),
+                   row(list(set(1), set(1))));
+
+        // Test Maps nested within Lists
+        assertRows(execute("SELECT [{}, (map<text, int>){'min' : min(ck), 'max' : max(ck)}] FROM %s"),
+                   row(list(map(), map("min", 1, "max", 3))));
+        assertRows(execute("SELECT [{}, (map<text, bigint>){'pk' : CAST(pk AS BIGINT), 'ck' : CAST(ck AS BIGINT), 'writetime' : WRITETIME(t)}] FROM %s"),
+                   row(list(map(), map("pk", 1L, "ck", 1L, "writetime", timestampInMicros))),
+                   row(list(map(), map("pk", 1L, "ck", 2L, "writetime", timestampInMicros))),
+                   row(list(map(), map("pk", 1L, "ck", 3L, "writetime", timestampInMicros))));
+        assertRows(execute("SELECT [{}, (map<text, int>){'pk' : pk, 'ck' : ck}] FROM %s WHERE pk = 1 ORDER BY ck DESC"),
+                   row(list(map(), map("pk", 1, "ck", 3))),
+                   row(list(map(), map("pk", 1, "ck", 2))),
+                   row(list(map(), map("pk", 1, "ck", 1))));
+
+        // Test Tuples nested within Lists
+        assertRows(execute("SELECT [(pk, ck, WRITETIME(t))] FROM %s"),
+                   row(list(tuple(1, 1, timestampInMicros))),
+                   row(list(tuple(1, 2, timestampInMicros))),
+                   row(list(tuple(1, 3, timestampInMicros))));
+        assertRows(execute("SELECT [(min(ck), max(ck))] FROM %s"),
+                   row(list(tuple(1, 3))));
+        assertRows(execute("SELECT [(CAST(pk AS BIGINT), CAST(ck AS BIGINT)), (t, WRITETIME(t))] FROM %s"),
+                   row(list(tuple(1L, 1L), tuple("one", timestampInMicros))),
+                   row(list(tuple(1L, 2L), tuple("two", timestampInMicros))),
+                   row(list(tuple(1L, 3L), tuple("three", timestampInMicros))));
+
+        // Test UDTs nested within Lists
+        String type = createType("CREATE TYPE %s(a int, b int, c bigint)");
+        assertRows(execute("SELECT [(" + type + "){a : min(ck), b: max(ck)}] FROM %s"),
+                   row(list(userType("a", 1, "b", 3, "c", null))));
+        assertRows(execute("SELECT [(" + type + "){a : pk, b : ck, c : WRITETIME(t)}] FROM %s"),
+                   row(list(userType("a", 1, "b", 1, "c", timestampInMicros))),
+                   row(list(userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(list(userType("a", 1, "b", 3, "c", timestampInMicros))));
+        assertRows(execute("SELECT [(" + type + "){a : pk, b : ck, c : WRITETIME(t)}] FROM %s WHERE pk = 1 ORDER BY ck DESC"),
+                   row(list(userType("a", 1, "b", 3, "c", timestampInMicros))),
+                   row(list(userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(list(userType("a", 1, "b", 1, "c", timestampInMicros))));
+
+        // Test Lists nested within Sets
+        assertRows(execute("SELECT {[], [min(ck), max(ck)]} FROM %s"),
+                   row(set(list(), list(1, 3))));
+        assertRows(execute("SELECT {[], [pk, ck]} FROM %s LIMIT 2"),
+                   row(set(list(), list(1, 1))),
+                   row(set(list(), list(1, 2))));
+        assertRows(execute("SELECT {[], [pk, ck]} FROM %s WHERE pk = 1 ORDER BY ck DESC LIMIT 2"),
+                   row(set(list(), list(1, 3))),
+                   row(set(list(), list(1, 2))));
+        assertRows(execute("SELECT {[min(ck)], ([max(ck)])} FROM %s"),
+                   row(set(list(1), list(3))));
+        assertRows(execute("SELECT {[pk], ([ck])} FROM %s"),
+                   row(set(list(1), list(1))),
+                   row(set(list(1), list(2))),
+                   row(set(list(1), list(3))));
+        assertRows(execute("SELECT {([min(ck)]), [max(ck)]} FROM %s"),
+                   row(set(list(1), list(3))));
+
+        // Test Sets nested within Sets
+        assertRows(execute("SELECT {{}, {min(ck), max(ck)}} FROM %s"),
+                   row(set(set(), set(1, 3))));
+        assertRows(execute("SELECT {{}, {pk, ck}} FROM %s LIMIT 2"),
+                   row(set(set(), set(1, 1))),
+                   row(set(set(), set(1, 2))));
+        assertRows(execute("SELECT {{}, {pk, ck}} FROM %s WHERE pk = 1 ORDER BY ck DESC LIMIT 2"),
+                   row(set(set(), set(1, 3))),
+                   row(set(set(), set(1, 2))));
+        assertRows(execute("SELECT {{min(ck)}, ({max(ck)})} FROM %s"),
+                   row(set(set(1), set(3))));
+        assertRows(execute("SELECT {{pk}, ({ck})} FROM %s"),
+                   row(set(set(1), set(1))),
+                   row(set(set(1), set(2))),
+                   row(set(set(1), set(3))));
+        assertRows(execute("SELECT {({min(ck)}), {max(ck)}} FROM %s"),
+                   row(set(set(1), set(3))));
+
+        // Test Maps nested within Sets
+        assertRows(execute("SELECT {{}, (map<text, int>){'min' : min(ck), 'max' : max(ck)}} FROM %s"),
+                   row(set(map(), map("min", 1, "max", 3))));
+        assertRows(execute("SELECT {{}, (map<text, int>){'pk' : pk, 'ck' : ck}} FROM %s"),
+                   row(set(map(), map("pk", 1, "ck", 1))),
+                   row(set(map(), map("pk", 1, "ck", 2))),
+                   row(set(map(), map("pk", 1, "ck", 3))));
+
+        // Test Tuples nested within Sets
+        assertRows(execute("SELECT {(pk, ck, WRITETIME(t))} FROM %s"),
+                   row(set(tuple(1, 1, timestampInMicros))),
+                   row(set(tuple(1, 2, timestampInMicros))),
+                   row(set(tuple(1, 3, timestampInMicros))));
+        assertRows(execute("SELECT {(min(ck), max(ck))} FROM %s"),
+                   row(set(tuple(1, 3))));
+
+        // Test UDTs nested within Sets
+        assertRows(execute("SELECT {(" + type + "){a : min(ck), b: max(ck)}} FROM %s"),
+                   row(set(userType("a", 1, "b", 3, "c", null))));
+        assertRows(execute("SELECT {(" + type + "){a : pk, b : ck, c : WRITETIME(t)}} FROM %s"),
+                   row(set(userType("a", 1, "b", 1, "c", timestampInMicros))),
+                   row(set(userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(set(userType("a", 1, "b", 3, "c", timestampInMicros))));
+        assertRows(execute("SELECT {(" + type + "){a : pk, b : ck, c : WRITETIME(t)}} FROM %s WHERE pk = 1 ORDER BY ck DESC"),
+                   row(set(userType("a", 1, "b", 3, "c", timestampInMicros))),
+                   row(set(userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(set(userType("a", 1, "b", 1, "c", timestampInMicros))));
+
+        // Test Lists nested within Maps
+        assertRows(execute("SELECT (map<frozen<list<int>>, frozen<list<int>>>){[min(ck)]:[max(ck)]} FROM %s"),
+                   row(map(list(1), list(3))));
+        assertRows(execute("SELECT (map<frozen<list<int>>, frozen<list<int>>>){[pk]: [ck]} FROM %s"),
+                   row(map(list(1), list(1))),
+                   row(map(list(1), list(2))),
+                   row(map(list(1), list(3))));
+
+        // Test Sets nested within Maps
+        assertRows(execute("SELECT (map<frozen<set<int>>, frozen<set<int>>>){{min(ck)} : {max(ck)}} FROM %s"),
+                   row(map(set(1), set(3))));
+        assertRows(execute("SELECT (map<frozen<set<int>>, frozen<set<int>>>){{pk} : {ck}} FROM %s"),
+                   row(map(set(1), set(1))),
+                   row(map(set(1), set(2))),
+                   row(map(set(1), set(3))));
+
+        // Test Maps nested within Maps
+        assertRows(execute("SELECT (map<frozen<map<text, int>>, frozen<map<text, int>>>){{'min' : min(ck)} : {'max' : max(ck)}} FROM %s"),
+                   row(map(map("min", 1), map("max", 3))));
+        assertRows(execute("SELECT (map<frozen<map<text, int>>, frozen<map<text, int>>>){{'pk' : pk} : {'ck' : ck}} FROM %s"),
+                   row(map(map("pk", 1), map("ck", 1))),
+                   row(map(map("pk", 1), map("ck", 2))),
+                   row(map(map("pk", 1), map("ck", 3))));
+
+        // Test Tuples nested within Maps
+        assertRows(execute("SELECT (map<frozen<tuple<int, int>>, frozen<tuple<bigint>>>){(pk, ck) : (WRITETIME(t))} FROM %s"),
+                   row(map(tuple(1, 1), tuple(timestampInMicros))),
+                   row(map(tuple(1, 2), tuple(timestampInMicros))),
+                   row(map(tuple(1, 3), tuple(timestampInMicros))));
+        assertRows(execute("SELECT (map<frozen<tuple<int>> , frozen<tuple<int>>>){(min(ck)) : (max(ck))} FROM %s"),
+                   row(map(tuple(1), tuple(3))));
+
+        // Test UDTs nested within Maps
+        assertRows(execute("SELECT (map<int, frozen<" + type + ">>){ck : {a : min(ck), b: max(ck)}} FROM %s"),
+                   row(map(1, userType("a", 1, "b", 3, "c", null))));
+        assertRows(execute("SELECT (map<int, frozen<" + type + ">>){ck : {a : pk, b : ck, c : WRITETIME(t)}} FROM %s"),
+                   row(map(1, userType("a", 1, "b", 1, "c", timestampInMicros))),
+                   row(map(2, userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(map(3, userType("a", 1, "b", 3, "c", timestampInMicros))));
+        assertRows(execute("SELECT (map<int, frozen<" + type + ">>){ck : {a : pk, b : ck, c : WRITETIME(t)}} FROM %s WHERE pk = 1 ORDER BY ck DESC"),
+                   row(map(3, userType("a", 1, "b", 3, "c", timestampInMicros))),
+                   row(map(2, userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(map(1, userType("a", 1, "b", 1, "c", timestampInMicros))));
+
+        // Test Lists nested within Tuples
+        assertRows(execute("SELECT ([min(ck)], [max(ck)]) FROM %s"),
+                   row(tuple(list(1), list(3))));
+        assertRows(execute("SELECT ([pk], [ck]) FROM %s"),
+                   row(tuple(list(1), list(1))),
+                   row(tuple(list(1), list(2))),
+                   row(tuple(list(1), list(3))));
+
+        // Test Sets nested within Tuples
+        assertRows(execute("SELECT ({min(ck)}, {max(ck)}) FROM %s"),
+                   row(tuple(set(1), set(3))));
+        assertRows(execute("SELECT ({pk}, {ck}) FROM %s"),
+                   row(tuple(set(1), set(1))),
+                   row(tuple(set(1), set(2))),
+                   row(tuple(set(1), set(3))));
+
+        // Test Maps nested within Tuples
+        assertRows(execute("SELECT ((map<text, int>){'min' : min(ck)}, (map<text, int>){'max' : max(ck)}) FROM %s"),
+                   row(tuple(map("min", 1), map("max", 3))));
+        assertRows(execute("SELECT ((map<text, int>){'pk' : pk}, (map<text, int>){'ck' : ck}) FROM %s"),
+                   row(tuple(map("pk", 1), map("ck", 1))),
+                   row(tuple(map("pk", 1), map("ck", 2))),
+                   row(tuple(map("pk", 1), map("ck", 3))));
+
+        // Test Tuples nested within Tuples
+        assertRows(execute("SELECT (tuple<tuple<int, int, bigint>>)((pk, ck, WRITETIME(t))) FROM %s"),
+                   row(tuple(tuple(1, 1, timestampInMicros))),
+                   row(tuple(tuple(1, 2, timestampInMicros))),
+                   row(tuple(tuple(1, 3, timestampInMicros))));
+        assertRows(execute("SELECT (tuple<tuple<int, int, bigint>>)((min(ck), max(ck))) FROM %s"),
+                   row(tuple(tuple(1, 3))));
+
+        assertRows(execute("SELECT ((t, WRITETIME(t)), (CAST(pk AS BIGINT), CAST(ck AS BIGINT))) FROM %s"),
+                   row(tuple(tuple("one", timestampInMicros), tuple(1L, 1L))),
+                   row(tuple(tuple("two", timestampInMicros), tuple(1L, 2L))),
+                   row(tuple(tuple("three", timestampInMicros), tuple(1L, 3L))));
+
+        // Test UDTs nested within Tuples
+        assertRows(execute("SELECT (tuple<" + type + ">)({a : min(ck), b: max(ck)}) FROM %s"),
+                   row(tuple(userType("a", 1, "b", 3, "c", null))));
+        assertRows(execute("SELECT (tuple<" + type + ">)({a : pk, b : ck, c : WRITETIME(t)}) FROM %s"),
+                   row(tuple(userType("a", 1, "b", 1, "c", timestampInMicros))),
+                   row(tuple(userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(tuple(userType("a", 1, "b", 3, "c", timestampInMicros))));
+        assertRows(execute("SELECT (tuple<" + type + ">)({a : pk, b : ck, c : WRITETIME(t)}) FROM %s WHERE pk = 1 ORDER BY ck DESC"),
+                   row(tuple(userType("a", 1, "b", 3, "c", timestampInMicros))),
+                   row(tuple(userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(tuple(userType("a", 1, "b", 1, "c", timestampInMicros))));
+
+        // Test Lists nested within UDTs
+        String containerType = createType("CREATE TYPE %s(l list<int>)");
+        assertRows(execute("SELECT (" + containerType + "){l : [min(ck), max(ck)]} FROM %s"),
+                   row(userType("l", list(1, 3))));
+        assertRows(execute("SELECT (" + containerType + "){l : [pk, ck]} FROM %s"),
+                   row(userType("l", list(1, 1))),
+                   row(userType("l", list(1, 2))),
+                   row(userType("l", list(1, 3))));
+
+        // Test Sets nested within UDTs
+        containerType = createType("CREATE TYPE %s(s set<int>)");
+        assertRows(execute("SELECT (" + containerType + "){s : {min(ck), max(ck)}} FROM %s"),
+                   row(userType("s", set(1, 3))));
+        assertRows(execute("SELECT (" + containerType + "){s : {pk, ck}} FROM %s"),
+                   row(userType("s", set(1))),
+                   row(userType("s", set(1, 2))),
+                   row(userType("s", set(1, 3))));
+
+        // Test Maps nested within UDTs
+        containerType = createType("CREATE TYPE %s(m map<text, int>)");
+        assertRows(execute("SELECT (" + containerType + "){m : {'min' : min(ck), 'max' : max(ck)}} FROM %s"),
+                   row(userType("m", map("min", 1, "max", 3))));
+        assertRows(execute("SELECT (" + containerType + "){m : {'pk' : pk, 'ck' : ck}} FROM %s"),
+                   row(userType("m", map("pk", 1, "ck", 1))),
+                   row(userType("m", map("pk", 1, "ck", 2))),
+                   row(userType("m", map("pk", 1, "ck", 3))));
+
+        // Test Tuples nested within UDTs
+        containerType = createType("CREATE TYPE %s(t tuple<int, int>, w tuple<bigint>)");
+        assertRows(execute("SELECT (" + containerType + "){t : (pk, ck), w : (WRITETIME(t))} FROM %s"),
+                   row(userType("t", tuple(1, 1), "w", tuple(timestampInMicros))),
+                   row(userType("t", tuple(1, 2), "w", tuple(timestampInMicros))),
+                   row(userType("t", tuple(1, 3), "w", tuple(timestampInMicros))));
+
+        // Test UDTs nested within Maps
+        containerType = createType("CREATE TYPE %s(t frozen<" + type + ">)");
+        assertRows(execute("SELECT (" + containerType + "){t : {a : min(ck), b: max(ck)}} FROM %s"),
+                   row(userType("t", userType("a", 1, "b", 3, "c", null))));
+        assertRows(execute("SELECT (" + containerType + "){t : {a : pk, b : ck, c : WRITETIME(t)}} FROM %s"),
+                   row(userType("t", userType("a", 1, "b", 1, "c", timestampInMicros))),
+                   row(userType("t", userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(userType("t", userType("a", 1, "b", 3, "c", timestampInMicros))));
+        assertRows(execute("SELECT (" + containerType + "){t : {a : pk, b : ck, c : WRITETIME(t)}} FROM %s WHERE pk = 1 ORDER BY ck DESC"),
+                   row(userType("t", userType("a", 1, "b", 3, "c", timestampInMicros))),
+                   row(userType("t", userType("a", 1, "b", 2, "c", timestampInMicros))),
+                   row(userType("t", userType("a", 1, "b", 1, "c", timestampInMicros))));
+
+
+        // Test Litteral Set with Duration elements
+        assertInvalidMessage("Durations are not allowed inside sets: set<duration>",
+                             "SELECT pk, ck, (set<duration>){2d, 1mo} FROM %s");
+
+        assertInvalidMessage("Invalid field selection: system.min(ck) of type int is not a user type",
+                             "SELECT min(ck).min FROM %s");
+        assertInvalidMessage("Invalid field selection: (map<text, int>){'min': system.min(ck), 'max': system.max(ck)} of type frozen<map<text, int>> is not a user type",
+                             "SELECT (map<text, int>) {'min' : min(ck), 'max' : max(ck)}.min FROM %s");
+    }
+
+    @Test
+    public void testCollectionLiteralsWithDurations() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, ck int, d1 duration, d2 duration, PRIMARY KEY (pk, ck) )");
+        execute("INSERT INTO %s (pk, ck, d1, d2) VALUES (1, 1, 15h, 13h)");
+        execute("INSERT INTO %s (pk, ck, d1, d2) VALUES (1, 2, 10h, 12h)");
+        execute("INSERT INTO %s (pk, ck, d1, d2) VALUES (1, 3, 11h, 13h)");
+
+        assertRows(execute("SELECT [d1, d2] FROM %s"),
+                   row(list(Duration.from("15h"), Duration.from("13h"))),
+                   row(list(Duration.from("10h"), Duration.from("12h"))),
+                   row(list(Duration.from("11h"), Duration.from("13h"))));
+
+        assertInvalidMessage("Durations are not allowed inside sets: frozen<set<duration>>", "SELECT {d1, d2} FROM %s");
+
+        assertRows(execute("SELECT (map<int, duration>){ck : d1} FROM %s"),
+                   row(map(1, Duration.from("15h"))),
+                   row(map(2, Duration.from("10h"))),
+                   row(map(3, Duration.from("11h"))));
+
+        assertInvalidMessage("Durations are not allowed as map keys: map<duration, int>",
+                             "SELECT (map<duration, int>){d1 : ck, d2 :ck} FROM %s");
     }
 
     @Test
@@ -81,11 +419,41 @@ public class TermSelectionTest extends CQLTester
 
         execute("INSERT INTO %s(k, v) VALUES (?, ?)", 0, userType("a", 3, "b", "foo"));
 
-        assertInvalidMessage("Cannot infer type for term", "SELECT k, v, { a: 4, b: 'bar'} FROM %s");
+        assertInvalidMessage("Cannot infer type for term", "SELECT { a: 4, b: 'bar'} FROM %s");
 
         assertRows(execute("SELECT k, v, (" + type + "){ a: 4, b: 'bar'} FROM %s"),
             row(0, userType("a", 3, "b", "foo"), userType("a", 4, "b", "bar"))
         );
+
+        assertRows(execute("SELECT k, v, (" + type + ")({ a: 4, b: 'bar'}) FROM %s"),
+            row(0, userType("a", 3, "b", "foo"), userType("a", 4, "b", "bar"))
+        );
+
+        assertRows(execute("SELECT k, v, ((" + type + "){ a: 4, b: 'bar'}).a FROM %s"),
+                   row(0, userType("a", 3, "b", "foo"), 4)
+        );
+
+        assertRows(execute("SELECT k, v, (" + type + "){ a: 4, b: 'bar'}.a FROM %s"),
+                   row(0, userType("a", 3, "b", "foo"), 4)
+        );
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT { a: 4} FROM %s");
+
+        assertRows(execute("SELECT k, v, (" + type + "){ a: 4} FROM %s"),
+            row(0, userType("a", 3, "b", "foo"), userType("a", 4, "b", null))
+        );
+
+        assertRows(execute("SELECT k, v, (" + type + "){ b: 'bar'} FROM %s"),
+                   row(0, userType("a", 3, "b", "foo"), userType("a", null, "b", "bar"))
+        );
+
+        execute("INSERT INTO %s(k, v) VALUES (?, ?)", 1, userType("a", 5, "b", "foo"));
+        assertRows(execute("SELECT (" + type + "){ a: max(v.a) , b: 'max'} FROM %s"),
+                   row(userType("a", 5, "b", "max"))
+        );
+        assertRows(execute("SELECT (" + type + "){ a: min(v.a) , b: 'min'} FROM %s"),
+                   row(userType("a", 3, "b", "min"))
+        );
     }
 
     @Test
@@ -221,11 +589,11 @@ public class TermSelectionTest extends CQLTester
         createFunctionOverload(fAmbiguousFunc1, "int,int",
                                                 "CREATE FUNCTION %s (val1 int, val2 int) " +
                                                 "CALLED ON NULL INPUT " +
-                                                "RETURNS bigint " +
+                                                "RETURNS int " +
                                                 "LANGUAGE java\n" +
-                                                "AS 'return (long)Math.max(val1, val2);';");
-        assertInvalidMessage("Ambiguous call to function cql_test_keyspace.function_",
-                             "SELECT pk, " + fAmbiguousFunc1 + "(valInt, 100) FROM %s");
+                                                "AS 'return Math.max(val1, val2);';");
+        assertRows(execute("SELECT pk, " + fAmbiguousFunc1 + "(valInt, 100) FROM %s"),
+                   row(1, 100));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index f167955..23a5a45 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -4478,4 +4478,14 @@ public class SelectTest extends CQLTester
             });
         }
     }
+
+    @Test
+    public void testWithDistinctAndJsonAsColumnName() throws Throwable
+    {
+        createTable("CREATE TABLE %s (distinct int, json int, value int, PRIMARY KEY(distinct, json))");
+        execute("INSERT INTO %s (distinct, json, value) VALUES (0, 0, 0)");
+
+        assertRows(execute("SELECT distinct, json FROM %s"), row(0, 0));
+        assertRows(execute("SELECT distinct distinct FROM %s"), row(0));
+    }
 }


[5/5] cassandra git commit: Merge branch cassandra-3.X into trunk

Posted by bl...@apache.org.
Merge branch cassandra-3.X into trunk


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

Branch: refs/heads/trunk
Commit: f782f148bdde962adf11d268db3435231dba5083
Parents: 58cf4c9 8b3de2f
Author: Benjamin Lerer <b....@gmail.com>
Authored: Mon Nov 21 18:54:34 2016 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Mon Nov 21 18:54:34 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   5 +
 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, 4769 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f782f148/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index fa9233a,1f1625c..6cd725c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,10 -1,8 +1,15 @@@
 +4.0
 + * Add column definition kind to dropped columns in schema (CASSANDRA-12705)
 + * Add (automate) Nodetool Documentation (CASSANDRA-12672)
 + * Update bundled cqlsh python driver to 3.7.0 (CASSANDRA-12736)
 + * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 + * Clean up the SSTableReader#getScanner API wrt removal of RateLimiter (CASSANDRA-12422)
 +
+ 3.12
+  * Add support for arithmetic operators (CASSANDRA-11935)
+ 
+ 3.11
+  * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
  
  3.10
   * Don't shut down socket input/output on StreamSession (CASSANDRA-12903)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f782f148/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------


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

Posted by bl...@apache.org.
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());
         }
     }
 


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

Posted by bl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java b/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
new file mode 100644
index 0000000..1f115a9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
@@ -0,0 +1,380 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.OperationExecutionException;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * Operation functions (Mathematics).
+ *
+ */
+public final class OperationFcts
+{
+    private static enum OPERATION
+    {
+        ADDITION('+', "_add")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.add(leftType, left, rightType, right);
+            }
+        },
+        SUBSTRACTION('-', "_substract")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.substract(leftType, left, rightType, right);
+            }
+        },
+        MULTIPLICATION('*', "_multiply")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.multiply(leftType, left, rightType, right);
+            }
+        },
+        DIVISION('/', "_divide")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.divide(leftType, left, rightType, right);
+            }
+        },
+        MODULO('%', "_modulo")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.mod(leftType, left, rightType, right);
+            }
+        };
+
+        /**
+         * The operator symbol.
+         */
+        private final char symbol;
+
+        /**
+         * The name of the function associated to this operation
+         */
+        private final String functionName;
+
+        private OPERATION(char symbol, String functionName)
+        {
+            this.symbol = symbol;
+            this.functionName = functionName;
+        }
+
+        /**
+         * Executes the operation between the specified operand.
+         *
+         * @param resultType the result ype of the operation
+         * @param leftType the type of the left operand
+         * @param left the left operand
+         * @param rightType the type of the right operand
+         * @param right the right operand
+         * @return the operation result
+         */
+        protected abstract ByteBuffer execute(NumberType<?> resultType,
+                                              NumberType<?> leftType,
+                                              ByteBuffer left,
+                                              NumberType<?> rightType,
+                                              ByteBuffer right);
+
+        /**
+         * Returns the {@code OPERATOR} associated to the specified function.
+         * @param functionName the function name
+         * @return the {@code OPERATOR} associated to the specified function
+         */
+        public static OPERATION fromFunctionName(String functionName)
+        {
+            for (OPERATION operator : values())
+            {
+                if (operator.functionName.equals(functionName))
+                    return operator;
+            }
+            return null;
+        }
+
+        /**
+         * Returns the {@code OPERATOR} with the specified symbol.
+         * @param functionName the function name
+         * @return the {@code OPERATOR} with the specified symbol
+         */
+        public static OPERATION fromSymbol(char symbol)
+        {
+            for (OPERATION operator : values())
+            {
+                if (operator.symbol == symbol)
+                    return operator;
+            }
+            return null;
+        }
+    }
+
+    /**
+     * The name of the function used to perform negations
+     */
+    public static final String NEGATION_FUNCTION_NAME = "_negate";
+
+    public static Collection<Function> all()
+    {
+        List<Function> functions = new ArrayList<>();
+
+        final NumberType<?>[] numericTypes = new NumberType[] { ByteType.instance,
+                                                                ShortType.instance,
+                                                                Int32Type.instance,
+                                                                LongType.instance,
+                                                                FloatType.instance,
+                                                                DoubleType.instance,
+                                                                DecimalType.instance,
+                                                                IntegerType.instance,
+                                                                CounterColumnType.instance};
+
+        for (NumberType<?> left : numericTypes)
+        {
+            for (NumberType<?> right : numericTypes)
+            {
+                NumberType<?> returnType = returnType(left, right);
+                functions.add(new OperationFunction(returnType, left, OPERATION.ADDITION, right));
+                functions.add(new OperationFunction(returnType, left, OPERATION.SUBSTRACTION, right));
+                functions.add(new OperationFunction(returnType, left, OPERATION.MULTIPLICATION, right));
+                functions.add(new OperationFunction(returnType, left, OPERATION.DIVISION, right));
+                functions.add(new OperationFunction(returnType, left, OPERATION.MODULO, right));
+            }
+            functions.add(new NegationFunction(left));
+        }
+        return functions;
+    }
+
+    /**
+     * Checks if the function with the specified name is an operation.
+     *
+     * @param function the function name
+     * @return {@code true} if the function is an operation, {@code false} otherwise.
+     */
+    public static boolean isOperation(FunctionName function)
+    {
+        return SchemaConstants.SYSTEM_KEYSPACE_NAME.equals(function.keyspace)
+                && OPERATION.fromFunctionName(function.name) != null;
+    }
+
+    /**
+     * Checks if the function with the specified name is a negation.
+     *
+     * @param function the function name
+     * @return {@code true} if the function is an negation, {@code false} otherwise.
+     */
+    public static boolean isNegation(FunctionName function)
+    {
+        return SchemaConstants.SYSTEM_KEYSPACE_NAME.equals(function.keyspace)&& NEGATION_FUNCTION_NAME.equals(function.name);
+    }
+
+    /**
+     * Returns the operator associated to the specified function.
+     *
+     * @return the operator associated to the specified function.
+     */
+    public static char getOperator(FunctionName function)
+    {
+        assert SchemaConstants.SYSTEM_KEYSPACE_NAME.equals(function.keyspace);
+        return OPERATION.fromFunctionName(function.name).symbol;
+    }
+
+    /**
+     * Returns the name of the function associated to the specified operator.
+     *
+     * @param operator the operator
+     * @return the name of the function associated to the specified operator
+     */
+    public static FunctionName getFunctionNameFromOperator(char operator)
+    {
+        return FunctionName.nativeFunction(OPERATION.fromSymbol(operator).functionName);
+    }
+
+    /**
+     * Determine the return type for an operation between the specified types.
+     *
+     * @param left the type of the left operand
+     * @param right the type of the right operand
+     * @return the return type for an operation between the specified types
+     */
+    private static NumberType<?> returnType(NumberType<?> left, NumberType<?> right)
+    {
+        boolean isFloatingPoint = left.isFloatingPoint() || right.isFloatingPoint();
+        int size = Math.max(size(left), size(right));
+        return isFloatingPoint
+             ? floatPointType(size)
+             : integerType(size);
+    }
+
+    /**
+     * Returns the number of bytes used to represent a value of this type.
+     * @return the number of bytes used to represent a value of this type or {@code Integer.MAX} if the number of bytes
+     * is not limited.
+     */
+    private static int size(NumberType<?> type)
+    {
+        int size = type.valueLengthIfFixed();
+
+        if (size > 0)
+            return size;
+
+        if (type.isCounter())
+            return LongType.instance.valueLengthIfFixed();
+
+        return Integer.MAX_VALUE;
+    }
+
+    private static NumberType<?> floatPointType(int size)
+    {
+        switch (size)
+        {
+            case 4: return FloatType.instance;
+            case 8: return DoubleType.instance;
+            default: return DecimalType.instance;
+        }
+    }
+
+    private static NumberType<?> integerType(int size)
+    {
+        switch (size)
+        {
+            case 1: return ByteType.instance;
+            case 2: return ShortType.instance;
+            case 4: return Int32Type.instance;
+            case 8: return LongType.instance;
+            default: return IntegerType.instance;
+        }
+    }
+
+    /**
+     * The class must not be instantiated.
+     */
+    private OperationFcts()
+    {
+    }
+
+    /**
+     * Function that execute operations.
+     */
+    private static class OperationFunction extends NativeScalarFunction
+    {
+        private final OPERATION operation;
+
+        public OperationFunction(NumberType<?> returnType,
+                                 NumberType<?> left,
+                                 OPERATION operation,
+                                 NumberType<?> right)
+        {
+            super(operation.functionName, returnType, left, right);
+            this.operation = operation;
+        }
+
+        @Override
+        public final String columnName(List<String> columnNames)
+        {
+            return String.format("%s %s %s", columnNames.get(0), getOperator(), columnNames.get(1));
+        }
+
+        public final ByteBuffer execute(ProtocolVersion protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer left = parameters.get(0);
+            ByteBuffer right = parameters.get(1);
+            if (left == null || !left.hasRemaining() || right == null || !right.hasRemaining())
+                return null;
+
+            NumberType<?> leftType = (NumberType<?>) argTypes().get(0);
+            NumberType<?> rightType = (NumberType<?>) argTypes().get(1);
+            NumberType<?> resultType = (NumberType<?>) returnType();
+
+            try
+            {
+                return operation.execute(resultType, leftType, left, rightType, right);
+            }
+            catch (Exception e)
+            {
+                throw OperationExecutionException.create(getOperator(), argTypes, e);
+            }
+        }
+
+        /**
+         * Returns the operator symbol.
+         * @return the operator symbol
+         */
+        private char getOperator()
+        {
+            return operation.symbol;
+        }
+    }
+
+    /**
+     * Function that negate a number.
+     */
+    private static class NegationFunction extends NativeScalarFunction
+    {
+        public NegationFunction(NumberType<?> inputType)
+        {
+            super(NEGATION_FUNCTION_NAME, inputType, inputType);
+        }
+
+        @Override
+        public final String columnName(List<String> columnNames)
+        {
+            return String.format("-%s", columnNames.get(0));
+        }
+
+        public final ByteBuffer execute(ProtocolVersion protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer input = parameters.get(0);
+            if (input == null)
+                return null;
+
+            NumberType<?> inputType = (NumberType<?>) argTypes().get(0);
+
+            return inputType.negate(input);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java b/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
new file mode 100644
index 0000000..de5fd93
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+/**
+ * A base <code>Selector.Factory</code> for collections or tuples.
+ */
+abstract class CollectionFactory extends Factory
+{
+    /**
+     * The collection or tuple type.
+     */
+    private final AbstractType<?> type;
+
+    /**
+     * The collection or tuple element factories.
+     */
+    private final SelectorFactories factories;
+
+    public CollectionFactory(AbstractType<?> type, SelectorFactories factories)
+    {
+        this.type = type;
+        this.factories = factories;
+    }
+
+    protected final AbstractType<?> getReturnType()
+    {
+        return type;
+    }
+
+    @Override
+    public final void addFunctionsTo(List<Function> functions)
+    {
+        factories.addFunctionsTo(functions);
+    }
+
+    @Override
+    public final boolean isAggregateSelectorFactory()
+    {
+        return factories.doesAggregation();
+    }
+
+    @Override
+    public final boolean isWritetimeSelectorFactory()
+    {
+        return factories.containsWritetimeSelectorFactory();
+    }
+
+    @Override
+    public final boolean isTTLSelectorFactory()
+    {
+        return factories.containsTTLSelectorFactory();
+    }
+
+    protected final void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+    {
+        SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
+        for (Factory factory : factories)
+           factory.addColumnMapping(tmpMapping, resultsColumn);
+
+        if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
+            // add a null mapping for cases where the collection is empty
+            mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+        else
+            // collate the mapped columns from the child factories & add those
+            mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java b/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
new file mode 100644
index 0000000..50b3ded
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.util.List;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A <code>Selector.Factory</code> which forwards all its method calls to another factory.
+ * Subclasses should override one or more methods to modify the behavior of the backing factory as desired per
+ * the decorator pattern.
+ */
+abstract class ForwardingFactory extends Factory
+{
+    /**
+     * Returns the backing delegate instance that methods are forwarded to.
+     */
+    protected abstract Factory delegate();
+
+    public Selector newInstance(QueryOptions options) throws InvalidRequestException
+    {
+        return delegate().newInstance(options);
+    }
+
+    protected String getColumnName()
+    {
+        return delegate().getColumnName();
+    }
+
+    protected AbstractType<?> getReturnType()
+    {
+        return delegate().getReturnType();
+    }
+
+    protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+    {
+        delegate().addColumnMapping(mapping, resultsColumn);
+    }
+
+    @Override
+    public void addFunctionsTo(List<Function> functions)
+    {
+        delegate().addFunctionsTo(functions);
+    }
+
+    @Override
+    public boolean isAggregateSelectorFactory()
+    {
+        return delegate().isAggregateSelectorFactory();
+    }
+
+    @Override
+    public boolean isWritetimeSelectorFactory()
+    {
+        return delegate().isWritetimeSelectorFactory();
+    }
+
+    @Override
+    public boolean isTTLSelectorFactory()
+    {
+        return delegate().isTTLSelectorFactory();
+    }
+
+    @Override
+    public boolean isSimpleSelectorFactory(int index)
+    {
+        return delegate().isSimpleSelectorFactory(index);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ListSelector.java b/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
new file mode 100644
index 0000000..940bd9c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * <code>Selector</code> for literal list (e.g. [min(value), max(value), count(value)]).
+ *
+ */
+final class ListSelector extends Selector
+{
+    /**
+     * The list type.
+     */
+    private final AbstractType<?> type;
+
+    /**
+     * The list elements
+     */
+    private final List<Selector> elements;
+
+    public static Factory newFactory(final AbstractType<?> type, final SelectorFactories factories)
+    {
+        return new CollectionFactory(type, factories)
+        {
+            protected String getColumnName()
+            {
+                return Lists.listToString(factories, Factory::getColumnName);
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                return new ListSelector(type, factories.newInstances(options));
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        List<ByteBuffer> buffers = new ArrayList<>(elements.size());
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            buffers.add(elements.get(i).getOutput(protocolVersion));
+        }
+        return CollectionSerializer.pack(buffers, buffers.size(), protocolVersion);
+    }
+
+    public void reset()
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).reset();
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return Lists.listToString(elements);
+    }
+
+    private ListSelector(AbstractType<?> type, List<Selector> elements)
+    {
+        this.type = type;
+        this.elements = elements;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/MapSelector.java b/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
new file mode 100644
index 0000000..8bbae8c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * <code>Selector</code> for literal map (e.g. {'min' : min(value), 'max' : max(value), 'count' : count(value)}).
+ *
+ */
+final class MapSelector extends Selector
+{
+    /**
+     * The map type.
+     */
+    private final MapType<?, ?> type;
+
+    /**
+     * The map elements
+     */
+    private final List<Pair<Selector, Selector>> elements;
+
+    public static Factory newFactory(final AbstractType<?> type, final List<Pair<Factory, Factory>> factories)
+    {
+        return new Factory()
+        {
+            protected String getColumnName()
+            {
+                return Maps.mapToString(factories, Factory::getColumnName);
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return type;
+            }
+
+            protected final void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+            {
+                SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    entry.left.addColumnMapping(tmpMapping, resultsColumn);
+                    entry.right.addColumnMapping(tmpMapping, resultsColumn);
+                }
+
+                if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
+                    // add a null mapping for cases where the collection is empty
+                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+                else
+                    // collate the mapped columns from the child factories & add those
+                    mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                return new MapSelector(type,
+                                        factories.stream()
+                                                 .map(p -> Pair.create(p.left.newInstance(options),
+                                                                       p.right.newInstance(options)))
+                                                 .collect(Collectors.toList()));
+            }
+
+            @Override
+            public boolean isAggregateSelectorFactory()
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    if (entry.left.isAggregateSelectorFactory() || entry.right.isAggregateSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+
+            @Override
+            public void addFunctionsTo(List<Function> functions)
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    entry.left.addFunctionsTo(functions);
+                    entry.right.addFunctionsTo(functions);
+                }
+            }
+
+            @Override
+            public boolean isWritetimeSelectorFactory()
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    if (entry.left.isWritetimeSelectorFactory() || entry.right.isWritetimeSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+
+            @Override
+            public boolean isTTLSelectorFactory()
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    if (entry.left.isTTLSelectorFactory() || entry.right.isTTLSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            Pair<Selector, Selector> pair = elements.get(i);
+            pair.left.addInput(protocolVersion, rs);
+            pair.right.addInput(protocolVersion, rs);
+        }
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        Map<ByteBuffer, ByteBuffer> map = new TreeMap<>(type.getKeysType());
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            Pair<Selector, Selector> pair = elements.get(i);
+            map.put(pair.left.getOutput(protocolVersion), pair.right.getOutput(protocolVersion));
+        }
+
+        List<ByteBuffer> buffers = new ArrayList<>(elements.size() * 2);
+        for (Map.Entry<ByteBuffer, ByteBuffer> entry : map.entrySet())
+        {
+            buffers.add(entry.getKey());
+            buffers.add(entry.getValue());
+        }
+        return CollectionSerializer.pack(buffers, elements.size(), protocolVersion);
+    }
+
+    public void reset()
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            Pair<Selector, Selector> pair = elements.get(i);
+            pair.left.reset();
+            pair.right.reset();
+        }
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return Maps.mapToString(elements);
+    }
+
+    private MapSelector(AbstractType<?> type, List<Pair<Selector, Selector>> elements)
+    {
+        this.type = (MapType<?, ?>) type;
+        this.elements = elements;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
index c05cdaa..15e0210 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
@@ -28,15 +28,6 @@ import org.apache.cassandra.transport.ProtocolVersion;
 
 final class ScalarFunctionSelector extends AbstractFunctionSelector<ScalarFunction>
 {
-    public boolean isAggregate()
-    {
-        // We cannot just return true as it is possible to have a scalar function wrapping an aggregation function
-        if (argSelectors.isEmpty())
-            return false;
-
-        return argSelectors.get(0).isAggregate();
-    }
-
     public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
     {
         for (int i = 0, m = argSelectors.size(); i < m; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/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 80e2ae8..20719ea 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -18,17 +18,22 @@
  */
 package org.apache.cassandra.cql3.selection;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
+import java.util.*;
+import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.text.StrBuilder;
+
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.Pair;
+
+import static org.apache.cassandra.cql3.selection.SelectorFactories.createFactoriesAndCollectColumnDefinitions;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 
 public interface Selectable extends AssignmentTestable
 {
@@ -154,7 +159,7 @@ public interface Selectable extends AssignmentTestable
         @Override
         public String toString()
         {
-            return rawTerm.toString();
+            return rawTerm.getText();
         }
 
         public static class Raw extends Selectable.Raw
@@ -244,11 +249,7 @@ public interface Selectable extends AssignmentTestable
         @Override
         public String toString()
         {
-            return new StrBuilder().append(function.name())
-                                   .append("(")
-                                   .appendWithSeparators(args, ", ")
-                                   .append(")")
-                                   .toString();
+            return function.columnName(args.stream().map(Object::toString).collect(Collectors.toList()));
         }
 
         public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
@@ -279,6 +280,18 @@ public interface Selectable extends AssignmentTestable
                                Collections.emptyList());
             }
 
+            public static Raw newOperation(char operator, Selectable.Raw left, Selectable.Raw right)
+            {
+                return new Raw(OperationFcts.getFunctionNameFromOperator(operator),
+                               Arrays.asList(left, right));
+            }
+
+            public static Raw newNegation(Selectable.Raw arg)
+            {
+                return new Raw(FunctionName.nativeFunction(OperationFcts.NEGATION_FUNCTION_NAME),
+                               Collections.singletonList(arg));
+            }
+
             public Selectable prepare(CFMetaData cfm)
             {
                 List<Selectable> preparedArgs = new ArrayList<>(args.size());
@@ -433,7 +446,16 @@ public interface Selectable extends AssignmentTestable
 
         public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
         {
-            Selector.Factory factory = selected.newSelectorFactory(cfm, null, defs, boundNames);
+            AbstractType<?> expectedUdtType = null;
+
+            // If the UDT is between parentheses, we know that it is not a tuple with a single element.
+            if (selected instanceof BetweenParenthesesOrWithTuple)
+            {
+                BetweenParenthesesOrWithTuple betweenParentheses = (BetweenParenthesesOrWithTuple) selected;
+                expectedUdtType = betweenParentheses.selectables.get(0).getExactTypeIfKnown(cfm.ksName);
+            }
+
+            Selector.Factory factory = selected.newSelectorFactory(cfm, expectedUdtType, defs, boundNames);
             AbstractType<?> type = factory.getColumnSpecification(cfm).type;
             if (!type.isUDT())
             {
@@ -485,4 +507,609 @@ public interface Selectable extends AssignmentTestable
             }
         }
     }
+
+    /**
+     * {@code Selectable} for {@code Selectable} between parentheses or tuples.
+     * <p>The parser cannot differentiate between a single element between parentheses or a single element tuple.
+     * By consequence, we are forced to wait until the type is known to be able to differentiate them.</p>
+     */
+    public static class BetweenParenthesesOrWithTuple implements Selectable
+    {
+        /**
+         * The tuple elements or the element between the parentheses
+         */
+        private final List<Selectable> selectables;
+
+        public BetweenParenthesesOrWithTuple(List<Selectable> selectables)
+        {
+            this.selectables = selectables;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            if (selectables.size() == 1 && !receiver.type.isTuple())
+                return selectables.get(0).testAssignment(keyspace, receiver);
+
+            return Tuples.testTupleAssignment(receiver, selectables);
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)",
+                                         this);
+            }
+
+            if (selectables.size() == 1 && !type.isTuple())
+                return newBetweenParenthesesSelectorFactory(cfm, expectedType, defs, boundNames);
+
+            return newTupleSelectorFactory(cfm, (TupleType) type, defs, boundNames);
+        }
+
+        private Factory newBetweenParenthesesSelectorFactory(CFMetaData cfm,
+                                                             AbstractType<?> expectedType,
+                                                             List<ColumnDefinition> defs,
+                                                             VariableSpecifications boundNames)
+        {
+            Selectable selectable = selectables.get(0);
+            final Factory factory = selectable.newSelectorFactory(cfm, expectedType, defs, boundNames);
+
+            return new ForwardingFactory()
+            {
+                protected Factory delegate()
+                {
+                    return factory;
+                }
+
+                protected String getColumnName()
+                {
+                    return String.format("(%s)", factory.getColumnName());
+                }
+            };
+        }
+
+        private Factory newTupleSelectorFactory(CFMetaData cfm,
+                                                TupleType tupleType,
+                                                List<ColumnDefinition> defs,
+                                                VariableSpecifications boundNames)
+        {
+            SelectorFactories factories = createFactoriesAndCollectColumnDefinitions(selectables,
+                                                                                     tupleType.allTypes(),
+                                                                                     cfm,
+                                                                                     defs,
+                                                                                     boundNames);
+
+            return TupleSelector.newFactory(tupleType, factories);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            // If there is only one element we cannot know if it is an element between parentheses or a tuple
+            // with only one element. By consequence, we need to force the user to specify the type.
+            if (selectables.size() == 1)
+                return null;
+
+            return Tuples.getExactTupleTypeIfKnown(selectables, p -> p.getExactTypeIfKnown(keyspace));
+        }
+
+        @Override
+        public String toString()
+        {
+            return Tuples.tupleToString(selectables);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final List<Selectable.Raw> raws;
+
+            public Raw(List<Selectable.Raw> raws)
+            {
+                this.raws = raws;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new BetweenParenthesesOrWithTuple(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
+            }
+        }
+    }
+
+    /**
+     * <code>Selectable</code> for literal Lists.
+     */
+    public static class WithList implements Selectable
+    {
+        /**
+         * The list elements
+         */
+        private final List<Selectable> selectables;
+
+        public WithList(List<Selectable> selectables)
+        {
+            this.selectables = selectables;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return Lists.testListAssignment(receiver, selectables);
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)",
+                                         this);
+            }
+
+            ListType<?> listType = (ListType<?>) type;
+
+            List<AbstractType<?>> expectedTypes = new ArrayList<>(selectables.size());
+            for (int i = 0, m = selectables.size(); i < m; i++)
+                expectedTypes.add(listType.getElementsType());
+
+            SelectorFactories factories = createFactoriesAndCollectColumnDefinitions(selectables,
+                                                                                     expectedTypes,
+                                                                                     cfm,
+                                                                                     defs,
+                                                                                     boundNames);
+            return ListSelector.newFactory(type, factories);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return Lists.getExactListTypeIfKnown(selectables, p -> p.getExactTypeIfKnown(keyspace));
+        }
+
+        @Override
+        public String toString()
+        {
+            return Lists.listToString(selectables);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final List<Selectable.Raw> raws;
+
+            public Raw(List<Selectable.Raw> raws)
+            {
+                this.raws = raws;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new WithList(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
+            }
+        }
+    }
+
+    /**
+     * <code>Selectable</code> for literal Sets.
+     */
+    public static class WithSet implements Selectable
+    {
+        /**
+         * The set elements
+         */
+        private final List<Selectable> selectables;
+
+        public WithSet(List<Selectable> selectables)
+        {
+            this.selectables = selectables;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return Sets.testSetAssignment(receiver, selectables);
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)",
+                                         this);
+            }
+
+            // The parser treats empty Maps as Sets so if the type is a MapType we know that the Map is empty
+            if (type instanceof MapType)
+                return MapSelector.newFactory(type, Collections.emptyList());
+
+            SetType<?> setType = (SetType<?>) type;
+
+            if (setType.getElementsType() == DurationType.instance)
+                throw invalidRequest("Durations are not allowed inside sets: %s", setType.asCQL3Type());
+
+            List<AbstractType<?>> expectedTypes = new ArrayList<>(selectables.size());
+            for (int i = 0, m = selectables.size(); i < m; i++)
+                expectedTypes.add(setType.getElementsType());
+
+            SelectorFactories factories = createFactoriesAndCollectColumnDefinitions(selectables,
+                                                                                     expectedTypes,
+                                                                                     cfm,
+                                                                                     defs,
+                                                                                     boundNames);
+
+            return SetSelector.newFactory(type, factories);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return Sets.getExactSetTypeIfKnown(selectables, p -> p.getExactTypeIfKnown(keyspace));
+        }
+
+        @Override
+        public String toString()
+        {
+            return Sets.setToString(selectables);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final List<Selectable.Raw> raws;
+
+            public Raw(List<Selectable.Raw> raws)
+            {
+                this.raws = raws;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new WithSet(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
+            }
+        }
+    }
+
+    /**
+     * {@code Selectable} for literal Maps or UDTs.
+     * <p>The parser cannot differentiate between a Map or a UDT in the selection cause because a
+     * {@code ColumnDefinition} is equivalent to a {@code FieldIdentifier} from a syntax point of view.
+     * By consequence, we are forced to wait until the type is known to be able to differentiate them.</p>
+     */
+    public static class WithMapOrUdt implements Selectable
+    {
+        /**
+         * The column family metadata. We need to store them to be able to build the proper data once the type has been
+         * identified.
+         */
+        private final CFMetaData cfm;
+
+        /**
+         * The Map or UDT raw elements.
+         */
+        private final List<Pair<Selectable.Raw, Selectable.Raw>> raws;
+
+        public WithMapOrUdt(CFMetaData cfm, List<Pair<Selectable.Raw, Selectable.Raw>> raws)
+        {
+            this.cfm = cfm;
+            this.raws = raws;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return receiver.type.isUDT() ? UserTypes.testUserTypeAssignment(receiver, getUdtFields((UserType) receiver.type))
+                                         : Maps.testMapAssignment(receiver, getMapEntries(cfm));
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)",
+                                         this);
+            }
+
+            if (type.isUDT())
+                return newUdtSelectorFactory(cfm, expectedType, defs, boundNames);
+
+            return newMapSelectorFactory(cfm, defs, boundNames, type);
+        }
+
+        private Factory newMapSelectorFactory(CFMetaData cfm,
+                                              List<ColumnDefinition> defs,
+                                              VariableSpecifications boundNames,
+                                              AbstractType<?> type)
+        {
+            MapType<?, ?> mapType = (MapType<?, ?>) type;
+
+            if (mapType.getKeysType() == DurationType.instance)
+                throw invalidRequest("Durations are not allowed as map keys: %s", mapType.asCQL3Type());
+
+            return MapSelector.newFactory(type, getMapEntries(cfm).stream()
+                                                                  .map(p -> Pair.create(p.left.newSelectorFactory(cfm, mapType.getKeysType(), defs, boundNames),
+                                                                                        p.right.newSelectorFactory(cfm, mapType.getValuesType(), defs, boundNames)))
+                                                                  .collect(Collectors.toList()));
+        }
+
+        private Factory newUdtSelectorFactory(CFMetaData cfm,
+                                              AbstractType<?> expectedType,
+                                              List<ColumnDefinition> defs,
+                                              VariableSpecifications boundNames)
+        {
+            UserType ut = (UserType) expectedType;
+            Map<FieldIdentifier, Factory> factories = new LinkedHashMap<>(ut.size());
+
+            for (Pair<Selectable.Raw, Selectable.Raw> raw : raws)
+            {
+                if (!(raw.left instanceof RawIdentifier))
+                    throw invalidRequest("%s is not a valid field identifier of type %s ",
+                                         raw.left,
+                                         ut.getNameAsString());
+
+                FieldIdentifier fieldName = ((RawIdentifier) raw.left).toFieldIdentifier();
+                int fieldPosition = ut.fieldPosition(fieldName);
+
+                if (fieldPosition == -1)
+                    throw invalidRequest("Unknown field '%s' in value of user defined type %s",
+                                         fieldName,
+                                         ut.getNameAsString());
+
+                AbstractType<?> fieldType = ut.fieldType(fieldPosition);
+                factories.put(fieldName,
+                              raw.right.prepare(cfm).newSelectorFactory(cfm, fieldType, defs, boundNames));
+            }
+
+            return UserTypeSelector.newFactory(expectedType, factories);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            // Lets force the user to specify the type.
+            return null;
+        }
+
+        @Override
+        public String toString()
+        {
+            return raws.stream()
+                       .map(p -> String.format("%s: %s",
+                                               p.left instanceof RawIdentifier ? p.left : p.left.prepare(cfm),
+                                               p.right.prepare(cfm)))
+                       .collect(Collectors.joining(", ", "{", "}"));
+        }
+
+        private List<Pair<Selectable, Selectable>> getMapEntries(CFMetaData cfm)
+        {
+            return raws.stream()
+                       .map(p -> Pair.create(p.left.prepare(cfm), p.right.prepare(cfm)))
+                       .collect(Collectors.toList());
+        }
+
+        private Map<FieldIdentifier, Selectable> getUdtFields(UserType ut)
+        {
+            Map<FieldIdentifier, Selectable> fields = new LinkedHashMap<>(ut.size());
+
+            for (Pair<Selectable.Raw, Selectable.Raw> raw : raws)
+            {
+                if (!(raw.left instanceof RawIdentifier))
+                    throw invalidRequest("%s is not a valid field identifier of type %s ",
+                                         raw.left,
+                                         ut.getNameAsString());
+
+                FieldIdentifier fieldName = ((RawIdentifier) raw.left).toFieldIdentifier();
+                int fieldPosition = ut.fieldPosition(fieldName);
+
+                if (fieldPosition == -1)
+                    throw invalidRequest("Unknown field '%s' in value of user defined type %s",
+                                         fieldName,
+                                         ut.getNameAsString());
+
+                fields.put(fieldName, raw.right.prepare(cfm));
+            }
+
+            return fields;
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final List<Pair<Selectable.Raw, Selectable.Raw>> raws;
+
+            public Raw(List<Pair<Selectable.Raw, Selectable.Raw>> raws)
+            {
+                this.raws = raws;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new WithMapOrUdt(cfm, raws);
+            }
+        }
+    }
+
+    /**
+     * <code>Selectable</code> for type hints (e.g. (int) ?).
+     */
+    public static class WithTypeHint implements Selectable
+    {
+
+        /**
+         * The name of the type as specified in the query.
+         */
+        private final String typeName;
+
+        /**
+         * The type specified by the hint.
+         */
+        private final AbstractType<?> type;
+
+       /**
+         * The selectable to which the hint should be applied.
+         */
+        private final Selectable selectable;
+
+        public WithTypeHint(String typeName, AbstractType<?> type, Selectable selectable)
+        {
+            this.typeName = typeName;
+            this.type = type;
+            this.selectable = selectable;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            if (receiver.type.equals(type))
+                return AssignmentTestable.TestResult.EXACT_MATCH;
+            else if (receiver.type.isValueCompatibleWith(type))
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+            else
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            final ColumnSpecification receiver = new ColumnSpecification(cfm.ksName, cfm.cfName, new ColumnIdentifier(toString(), true), type);
+
+            if (!selectable.testAssignment(cfm.ksName, receiver).isAssignable())
+                throw new InvalidRequestException(String.format("Cannot assign value %s to %s of type %s", this, receiver.name, receiver.type.asCQL3Type()));
+
+            final Factory factory = selectable.newSelectorFactory(cfm, type, defs, boundNames);
+
+            return new ForwardingFactory()
+            {
+                protected Factory delegate()
+                {
+                    return factory;
+                }
+
+                protected AbstractType<?> getReturnType()
+                {
+                    return type;
+                }
+
+                protected String getColumnName()
+                {
+                    return String.format("(%s)%s", typeName, factory.getColumnName());
+                }
+            };
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return type;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("(%s)%s", typeName, selectable);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final CQL3Type.Raw typeRaw;
+
+            private final Selectable.Raw raw;
+
+            public Raw( CQL3Type.Raw typeRaw, Selectable.Raw raw)
+            {
+                this.typeRaw = typeRaw;
+                this.raw = raw;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                Selectable selectable = raw.prepare(cfm);
+                AbstractType<?> type = this.typeRaw.prepare(cfm.ksName).getType();
+                if (type.isFreezable())
+                    type = type.freeze();
+                return new WithTypeHint(typeRaw.toString(), type, selectable);
+            }
+        }
+    }
+
+    /**
+     * In the selection clause, the parser cannot differentiate between Maps and UDTs as a column identifier and field
+     * identifier have the same syntax. By consequence, we need to wait until the type is known to create the proper
+     * Object: {@code ColumnDefinition} or {@code FieldIdentifier}.
+     */
+    public static final class RawIdentifier extends Selectable.Raw
+    {
+        private final String text;
+
+        private final boolean quoted;
+
+        /**
+         * Creates a {@code RawIdentifier} from an unquoted identifier string.
+         */
+        public static Raw forUnquoted(String text)
+        {
+            return new RawIdentifier(text, false);
+        }
+
+        /**
+         * Creates a {@code RawIdentifier} from a quoted identifier string.
+         */
+        public static Raw forQuoted(String text)
+        {
+            return new RawIdentifier(text, true);
+        }
+
+        private RawIdentifier(String text, boolean quoted)
+        {
+            this.text = text;
+            this.quoted = quoted;
+        }
+
+        @Override
+        public Selectable prepare(CFMetaData cfm)
+        {
+            ColumnDefinition.Raw raw = quoted ? ColumnDefinition.Raw.forQuoted(text)
+                                              : ColumnDefinition.Raw.forUnquoted(text);
+            return raw.prepare(cfm);
+        }
+
+        public FieldIdentifier toFieldIdentifier()
+        {
+            return quoted ? FieldIdentifier.forQuoted(text)
+                          : FieldIdentifier.forUnquoted(text);
+        }
+
+        @Override
+        public String toString()
+        {
+            return text;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/Selector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selector.java b/src/java/org/apache/cassandra/cql3/selection/Selector.java
index 922b57f..6f83dfc 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selector.java
@@ -172,17 +172,6 @@ public abstract class Selector
     public abstract AbstractType<?> getType();
 
     /**
-     * Checks if this <code>Selector</code> is creating aggregates.
-     *
-     * @return <code>true</code> if this <code>Selector</code> is creating aggregates <code>false</code>
-     * otherwise.
-     */
-    public boolean isAggregate()
-    {
-        return false;
-    }
-
-    /**
      * Reset the internal state of this <code>Selector</code>.
      */
     public abstract void reset();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SetSelector.java b/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
new file mode 100644
index 0000000..34de078
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * <code>Selector</code> for literal set (e.g. {min(value), max(value), count(value)}).
+ *
+ */
+final class SetSelector extends Selector
+{
+    /**
+     * The set type.
+     */
+    private final SetType<?> type;
+
+    /**
+     * The set elements
+     */
+    private final List<Selector> elements;
+
+    public static Factory newFactory(final AbstractType<?> type, final SelectorFactories factories)
+    {
+        return new CollectionFactory(type, factories)
+        {
+            protected String getColumnName()
+            {
+                return Sets.setToString(factories, Factory::getColumnName);
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                return new SetSelector(type, factories.newInstances(options));
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        Set<ByteBuffer> buffers = new TreeSet<>(type.getElementsType());
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            buffers.add(elements.get(i).getOutput(protocolVersion));
+        }
+        return CollectionSerializer.pack(buffers, buffers.size(), protocolVersion);
+    }
+
+    public void reset()
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).reset();
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return Sets.setToString(elements);
+    }
+
+    private SetSelector(AbstractType<?> type, List<Selector> elements)
+    {
+        this.type = (SetType<?>) type;
+        this.elements = elements;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/TupleSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/TupleSelector.java b/src/java/org/apache/cassandra/cql3/selection/TupleSelector.java
new file mode 100644
index 0000000..9f4c381
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/TupleSelector.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * <code>Selector</code> for literal tuples (e.g. (min(value), max(value), count(value))).
+ *
+ */
+final class TupleSelector extends Selector
+{
+    /**
+     * The tuple type.
+     */
+    private final AbstractType<?> type;
+
+    /**
+     * The tuple elements
+     */
+    private final List<Selector> elements;
+
+    public static Factory newFactory(final AbstractType<?> type, final SelectorFactories factories)
+    {
+        return new CollectionFactory(type, factories)
+        {
+            protected String getColumnName()
+            {
+                return Tuples.tupleToString(factories, Factory::getColumnName);
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                return new TupleSelector(type, factories.newInstances(options));
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        ByteBuffer[] buffers = new ByteBuffer[elements.size()];
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            buffers[i] = elements.get(i).getOutput(protocolVersion);
+        }
+        return TupleType.buildValue(buffers);
+    }
+
+    public void reset()
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).reset();
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return Tuples.tupleToString(elements);
+    }
+
+    private TupleSelector(AbstractType<?> type, List<Selector> elements)
+    {
+        this.type = type;
+        this.elements = elements;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java b/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
new file mode 100644
index 0000000..3c298b5
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * <code>Selector</code> for literal map (e.g. {'min' : min(value), 'max' : max(value), 'count' : count(value)}).
+ *
+ */
+final class UserTypeSelector extends Selector
+{
+    /**
+     * The map type.
+     */
+    private final AbstractType<?> type;
+
+    /**
+     * The user type fields
+     */
+    private final Map<FieldIdentifier, Selector> fields;
+
+    public static Factory newFactory(final AbstractType<?> type, final Map<FieldIdentifier, Factory> factories)
+    {
+        return new Factory()
+        {
+            protected String getColumnName()
+            {
+                return UserTypes.userTypeToString(factories, Factory::getColumnName);
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return type;
+            }
+
+            protected final void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+            {
+                SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
+                for (Factory factory : factories.values())
+                {
+                    factory.addColumnMapping(tmpMapping, resultsColumn);
+                }
+
+                if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
+                    // add a null mapping for cases where the collection is empty
+                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+                else
+                    // collate the mapped columns from the child factories & add those
+                    mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                Map<FieldIdentifier, Selector> fields = new HashMap<>(factories.size());
+                for (Entry<FieldIdentifier, Factory> factory : factories.entrySet())
+                    fields.put(factory.getKey(), factory.getValue().newInstance(options));
+
+                return new UserTypeSelector(type, fields);
+            }
+
+            @Override
+            public boolean isAggregateSelectorFactory()
+            {
+                for (Factory factory : factories.values())
+                {
+                    if (factory.isAggregateSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+
+            @Override
+            public void addFunctionsTo(List<Function> functions)
+            {
+                for (Factory factory : factories.values())
+                    factory.addFunctionsTo(functions);
+            }
+
+            @Override
+            public boolean isWritetimeSelectorFactory()
+            {
+                for (Factory factory : factories.values())
+                {
+                    if (factory.isWritetimeSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+
+            @Override
+            public boolean isTTLSelectorFactory()
+            {
+                for (Factory factory : factories.values())
+                {
+                    if (factory.isTTLSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (Selector field : fields.values())
+            field.addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        UserType userType = (UserType) type;
+        ByteBuffer[] buffers = new ByteBuffer[userType.size()];
+        for (int i = 0, m = userType.size(); i < m; i++)
+        {
+            Selector selector = fields.get(userType.fieldName(i));
+            if (selector != null)
+                buffers[i] = selector.getOutput(protocolVersion);
+        }
+        return TupleType.buildValue(buffers);
+    }
+
+    public void reset()
+    {
+        for (Selector field : fields.values())
+            field.reset();
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return UserTypes.userTypeToString(fields);
+    }
+
+    private UserTypeSelector(AbstractType<?> type, Map<FieldIdentifier, Selector> fields)
+    {
+        this.type = type;
+        this.fields = fields;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/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 7f6b6ab..31a461b 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -477,6 +477,7 @@ public final class SystemKeyspace
                         .add(BytesConversionFcts.all())
                         .add(AggregateFcts.all())
                         .add(CastFcts.all())
+                        .add(OperationFcts.all())
                         .build();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/AbstractType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 37a1959..99df8a2 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -319,6 +319,11 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         return false;
     }
 
+    public boolean isTuple()
+    {
+        return false;
+    }
+
     public boolean isMultiCell()
     {
         return false;
@@ -384,7 +389,7 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
     /**
      * The length of values for this type if all values are of fixed length, -1 otherwise.
      */
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return -1;
     }
@@ -487,9 +492,13 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
 
     public final AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
     {
+        return testAssignment(receiver.type);
+    }
+
+    public final AssignmentTestable.TestResult testAssignment(AbstractType<?> receiverType)
+    {
         // We should ignore the fact that the output type is frozen in our comparison as functions do not support
         // frozen types for arguments
-        AbstractType<?> receiverType = receiver.type;
         if (isFreezable() && !isMultiCell())
             receiverType = receiverType.freeze();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/BooleanType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/BooleanType.java b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
index 1dbd1af..475cae6 100644
--- a/src/java/org/apache/cassandra/db/marshal/BooleanType.java
+++ b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
@@ -97,7 +97,7 @@ public class BooleanType extends AbstractType<Boolean>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 1;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/ByteType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteType.java b/src/java/org/apache/cassandra/db/marshal/ByteType.java
index 55aea8f..517e734 100644
--- a/src/java/org/apache/cassandra/db/marshal/ByteType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ByteType.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class ByteType extends AbstractType<Byte>
+public class ByteType extends NumberType<Byte>
 {
     public static final ByteType instance = new ByteType();
 
@@ -84,8 +84,62 @@ public class ByteType extends AbstractType<Byte>
     }
 
     @Override
+    public int valueLengthIfFixed()
+    {
+        return 1;
+    }
+
+    @Override
     public TypeSerializer<Byte> getSerializer()
     {
         return ByteSerializer.instance;
     }
+
+    @Override
+    public byte toByte(ByteBuffer value)
+    {
+        return ByteBufferUtil.toByte(value);
+    }
+
+    @Override
+    public short toShort(ByteBuffer value)
+    {
+        return toByte(value);
+    }
+
+    @Override
+    protected int toInt(ByteBuffer value)
+    {
+        return toByte(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) + rightType.toByte(right)));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) - rightType.toByte(right)));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) * rightType.toByte(right)));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) / rightType.toByte(right)));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) % rightType.toByte(right)));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes((byte) -toByte(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
index 8bb1a25..8777e0e 100644
--- a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
@@ -22,13 +22,13 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.CounterSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class CounterColumnType extends AbstractType<Long>
+public class CounterColumnType extends NumberType<Long>
 {
     public static final CounterColumnType instance = new CounterColumnType();
 
@@ -93,4 +93,40 @@ public class CounterColumnType extends AbstractType<Long>
     {
         return CounterSerializer.instance;
     }
+
+    @Override
+    protected long toLong(ByteBuffer value)
+    {
+        return ByteBufferUtil.toLong(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) + rightType.toLong(right));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) - rightType.toLong(right));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) * rightType.toLong(right));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) / rightType.toLong(right));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) % rightType.toLong(right));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes(-toLong(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/DateType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DateType.java b/src/java/org/apache/cassandra/db/marshal/DateType.java
index 87b2cad..473cedf 100644
--- a/src/java/org/apache/cassandra/db/marshal/DateType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DateType.java
@@ -119,7 +119,7 @@ public class DateType extends AbstractType<Date>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 8;
     }


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

Posted by bl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/DecimalType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DecimalType.java b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
index f1586e0..b98bf00 100644
--- a/src/java/org/apache/cassandra/db/marshal/DecimalType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
@@ -18,6 +18,8 @@
 package org.apache.cassandra.db.marshal;
 
 import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.math.MathContext;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
@@ -29,7 +31,7 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class DecimalType extends AbstractType<BigDecimal>
+public class DecimalType extends NumberType<BigDecimal>
 {
     public static final DecimalType instance = new DecimalType();
 
@@ -40,6 +42,12 @@ public class DecimalType extends AbstractType<BigDecimal>
         return true;
     }
 
+    @Override
+    public boolean isFloatingPoint()
+    {
+        return true;
+    }
+
     public int compareCustom(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -95,4 +103,70 @@ public class DecimalType extends AbstractType<BigDecimal>
     {
         return DecimalSerializer.instance;
     }
+
+    @Override
+    protected int toInt(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected float toFloat(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected long toLong(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected double toDouble(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected BigInteger toBigInteger(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected BigDecimal toBigDecimal(ByteBuffer value)
+    {
+        return compose(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigDecimal(left).add(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigDecimal(left).subtract(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigDecimal(left).multiply(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigDecimal(left).divide(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigDecimal(left).remainder(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return decompose(toBigDecimal(input).negate());
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/DoubleType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DoubleType.java b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
index d2309ee..b72d3e9 100644
--- a/src/java/org/apache/cassandra/db/marshal/DoubleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class DoubleType extends AbstractType<Double>
+public class DoubleType extends NumberType<Double>
 {
     public static final DoubleType instance = new DoubleType();
 
@@ -39,6 +39,12 @@ public class DoubleType extends AbstractType<Double>
         return true;
     }
 
+    @Override
+    public boolean isFloatingPoint()
+    {
+        return true;
+    }
+
     public int compareCustom(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -53,17 +59,14 @@ public class DoubleType extends AbstractType<Double>
       if (source.isEmpty())
           return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 
-      Double d;
       try
       {
-          d = Double.valueOf(source);
+          return decompose(Double.valueOf(source));
       }
       catch (NumberFormatException e1)
       {
           throw new MarshalException(String.format("Unable to make double from '%s'", source), e1);
       }
-
-      return decompose(d);
     }
 
     @Override
@@ -100,8 +103,62 @@ public class DoubleType extends AbstractType<Double>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 8;
     }
+
+    @Override
+    protected int toInt(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected float toFloat(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected long toLong(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected double toDouble(ByteBuffer value)
+    {
+        return ByteBufferUtil.toDouble(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toDouble(left) + rightType.toDouble(right));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toDouble(left) - rightType.toDouble(right));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toDouble(left) * rightType.toDouble(right));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toDouble(left) / rightType.toDouble(right));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toDouble(left) % rightType.toDouble(right));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes(-toDouble(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/EmptyType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/EmptyType.java b/src/java/org/apache/cassandra/db/marshal/EmptyType.java
index c653084..87b3a7f 100644
--- a/src/java/org/apache/cassandra/db/marshal/EmptyType.java
+++ b/src/java/org/apache/cassandra/db/marshal/EmptyType.java
@@ -78,7 +78,7 @@ public class EmptyType extends AbstractType<Void>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 0;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/FloatType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/FloatType.java b/src/java/org/apache/cassandra/db/marshal/FloatType.java
index 5445bbc..33d8344 100644
--- a/src/java/org/apache/cassandra/db/marshal/FloatType.java
+++ b/src/java/org/apache/cassandra/db/marshal/FloatType.java
@@ -29,7 +29,7 @@ import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class FloatType extends AbstractType<Float>
+public class FloatType extends NumberType<Float>
 {
     public static final FloatType instance = new FloatType();
 
@@ -40,6 +40,12 @@ public class FloatType extends AbstractType<Float>
         return true;
     }
 
+    @Override
+    public boolean isFloatingPoint()
+    {
+        return true;
+    }
+
     public int compareCustom(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -56,8 +62,7 @@ public class FloatType extends AbstractType<Float>
 
       try
       {
-          float f = Float.parseFloat(source);
-          return ByteBufferUtil.bytes(f);
+          return decompose(Float.parseFloat(source));
       }
       catch (NumberFormatException e1)
       {
@@ -99,8 +104,56 @@ public class FloatType extends AbstractType<Float>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 4;
     }
+
+    @Override
+    protected int toInt(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected float toFloat(ByteBuffer value)
+    {
+        return ByteBufferUtil.toFloat(value);
+    }
+
+    @Override
+    protected double toDouble(ByteBuffer value)
+    {
+        return toFloat(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toFloat(left) + rightType.toFloat(right));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toFloat(left) - rightType.toFloat(right));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toFloat(left) * rightType.toFloat(right));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toFloat(left) / rightType.toFloat(right));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toFloat(left) % rightType.toFloat(right));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes(-toFloat(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/Int32Type.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/Int32Type.java b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
index 1c8c93e..a66f9dc 100644
--- a/src/java/org/apache/cassandra/db/marshal/Int32Type.java
+++ b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
@@ -22,13 +22,13 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.Constants;
 import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.Int32Serializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class Int32Type extends AbstractType<Integer>
+public class Int32Type extends NumberType<Integer>
 {
     public static final Int32Type instance = new Int32Type();
 
@@ -112,8 +112,50 @@ public class Int32Type extends AbstractType<Integer>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 4;
     }
+
+    @Override
+    protected int toInt(ByteBuffer value)
+    {
+        return ByteBufferUtil.toInt(value);
+    }
+
+    @Override
+    protected float toFloat(ByteBuffer value)
+    {
+        return toInt(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toInt(left) + rightType.toInt(right));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toInt(left) - rightType.toInt(right));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toInt(left) * rightType.toInt(right));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toInt(left) / rightType.toInt(right));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toInt(left) % rightType.toInt(right));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes(-toInt(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/IntegerType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/IntegerType.java b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
index 944a231..e2b8518 100644
--- a/src/java/org/apache/cassandra/db/marshal/IntegerType.java
+++ b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.db.marshal;
 
+import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
 
@@ -29,7 +30,7 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public final class IntegerType extends AbstractType<BigInteger>
+public final class IntegerType extends NumberType<BigInteger>
 {
     public static final IntegerType instance = new IntegerType();
 
@@ -184,4 +185,70 @@ public final class IntegerType extends AbstractType<BigInteger>
     {
         return IntegerSerializer.instance;
     }
+
+    @Override
+    protected int toInt(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected float toFloat(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected long toLong(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected double toDouble(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected BigInteger toBigInteger(ByteBuffer value)
+    {
+        return compose(value);
+    }
+
+    @Override
+    protected BigDecimal toBigDecimal(ByteBuffer value)
+    {
+        return new BigDecimal(compose(value));
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigInteger(left).add(rightType.toBigInteger(right)));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigInteger(left).subtract(rightType.toBigInteger(right)));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigInteger(left).multiply(rightType.toBigInteger(right)));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigInteger(left).divide(rightType.toBigInteger(right)));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return decompose(leftType.toBigInteger(left).remainder(rightType.toBigInteger(right)));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return decompose(toBigInteger(input).negate());
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
index 70767d4..de32a56 100644
--- a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
@@ -86,7 +86,7 @@ public class LexicalUUIDType extends AbstractType<UUID>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 16;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/LongType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/LongType.java b/src/java/org/apache/cassandra/db/marshal/LongType.java
index c852461..ef96e2e 100644
--- a/src/java/org/apache/cassandra/db/marshal/LongType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LongType.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class LongType extends AbstractType<Long>
+public class LongType extends NumberType<Long>
 {
     public static final LongType instance = new LongType();
 
@@ -120,8 +120,56 @@ public class LongType extends AbstractType<Long>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 8;
     }
+
+    @Override
+    protected int toInt(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected float toFloat(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected long toLong(ByteBuffer value)
+    {
+        return ByteBufferUtil.toLong(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) + rightType.toLong(right));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) - rightType.toLong(right));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) * rightType.toLong(right));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) / rightType.toLong(right));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) % rightType.toLong(right));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes(-toLong(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/NumberType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/NumberType.java b/src/java/org/apache/cassandra/db/marshal/NumberType.java
new file mode 100644
index 0000000..9e7697f
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/NumberType.java
@@ -0,0 +1,223 @@
+/*
+ * 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.db.marshal;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+/**
+ * Base type for the numeric types.
+ */
+public abstract class NumberType<T extends Number> extends AbstractType<T>
+{
+    protected NumberType(ComparisonType comparisonType)
+    {
+        super(comparisonType);
+    }
+
+    /**
+     * Checks if this type support floating point numbers.
+     * @return {@code true} if this type support floating point numbers, {@code false} otherwise.
+     */
+    public boolean isFloatingPoint()
+    {
+        return false;
+    }
+
+    /**
+     * Converts the specified value into a <code>BigInteger</code> if allowed.
+     *
+     * @param value the value to convert
+     * @return the converted value
+     * @throws UnsupportedOperationException if the value cannot be converted without losing precision
+     */
+    protected BigInteger toBigInteger(ByteBuffer value)
+    {
+        return BigInteger.valueOf(toLong(value));
+    }
+
+    /**
+     * Converts the specified value into a <code>BigDecimal</code>.
+     *
+     * @param value the value to convert
+     * @return the converted value
+     */
+    protected BigDecimal toBigDecimal(ByteBuffer value)
+    {
+        double d = toDouble(value);
+
+        if (Double.isNaN(d))
+            throw new NumberFormatException("A NaN cannot be converted into a decimal");
+
+        if (Double.isInfinite(d))
+            throw new NumberFormatException("An infinite number cannot be converted into a decimal");
+
+        return BigDecimal.valueOf(d);
+    }
+
+    /**
+     * Converts the specified value into a <code>byte</code> if allowed.
+     *
+     * @param value the value to convert
+     * @return the converted value
+     * @throws UnsupportedOperationException if the value cannot be converted without losing precision
+     */
+    protected byte toByte(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Converts the specified value into a <code>short</code> if allowed.
+     *
+     * @param value the value to convert
+     * @return the converted value
+     * @throws UnsupportedOperationException if the value cannot be converted without losing precision
+     */
+    protected short toShort(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Converts the specified value into an <code>int</code> if allowed.
+     *
+     * @param value the value to convert
+     * @return the converted value
+     * @throws UnsupportedOperationException if the value cannot be converted without losing precision
+     */
+    protected int toInt(ByteBuffer value)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Converts the specified value into a <code>long</code> if allowed.
+     *
+     * @param value the value to convert
+     * @return the converted value
+     * @throws UnsupportedOperationException if the value cannot be converted without losing precision
+     */
+    protected long toLong(ByteBuffer value)
+    {
+        return toInt(value);
+    }
+
+    /**
+     * Converts the specified value into a <code>float</code> if allowed.
+     *
+     * @param value the value to convert
+     * @return the converted value
+     * @throws UnsupportedOperationException if the value cannot be converted without losing precision
+     */
+    protected float toFloat(ByteBuffer value)
+    {
+        return toInt(value);
+    }
+
+    /**
+     * Converts the specified value into a <code>double</code> if allowed.
+     *
+     * @param value the value to convert
+     * @return the converted value
+     * @throws UnsupportedOperationException if the value cannot be converted without losing precision
+     */
+    protected double toDouble(ByteBuffer value)
+    {
+        return toLong(value);
+    }
+
+    /**
+     * Adds the left argument to the right one.
+     *
+     * @param leftType the type associated to the left argument
+     * @param left the left argument
+     * @param rightType the type associated to the right argument
+     * @param right the right argument
+     * @return the addition result
+     */
+    public abstract ByteBuffer add(NumberType<?> leftType,
+                                   ByteBuffer left,
+                                   NumberType<?> rightType,
+                                   ByteBuffer right);
+
+    /**
+     * Substracts the left argument from the right one.
+     *
+     * @param leftType the type associated to the left argument
+     * @param left the left argument
+     * @param rightType the type associated to the right argument
+     * @param right the right argument
+     * @return the substraction result
+     */
+    public abstract ByteBuffer substract(NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right);
+
+    /**
+     * Multiplies the left argument with the right one.
+     *
+     * @param leftType the type associated to the left argument
+     * @param left the left argument
+     * @param rightType the type associated to the right argument
+     * @param right the right argument
+     * @return the multiplication result
+     */
+    public abstract ByteBuffer multiply(NumberType<?> leftType,
+                                        ByteBuffer left,
+                                        NumberType<?> rightType,
+                                        ByteBuffer right);
+
+    /**
+     * Divides the left argument by the right one.
+     *
+     * @param leftType the type associated to the left argument
+     * @param left the left argument
+     * @param rightType the type associated to the right argument
+     * @param right the right argument
+     * @return the division result
+     */
+    public abstract ByteBuffer divide(NumberType<?> leftType,
+                                      ByteBuffer left,
+                                      NumberType<?> rightType,
+                                      ByteBuffer right);
+
+    /**
+     * Return the remainder.
+     *
+     * @param leftType the type associated to the left argument
+     * @param left the left argument
+     * @param rightType the type associated to the right argument
+     * @param right the right argument
+     * @return the remainder
+     */
+    public abstract ByteBuffer mod(NumberType<?> leftType,
+                                   ByteBuffer left,
+                                   NumberType<?> rightType,
+                                   ByteBuffer right);
+
+    /**
+     * Negates the argument.
+     *
+     * @param input the argument to negate
+     * @return the negated argument
+     */
+    public abstract ByteBuffer negate(ByteBuffer input);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/ReversedType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
index 0eb0046..250dfdc 100644
--- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
@@ -132,7 +132,7 @@ public class ReversedType<T> extends AbstractType<T>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return baseType.valueLengthIfFixed();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/ShortType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ShortType.java b/src/java/org/apache/cassandra/db/marshal/ShortType.java
index 7645ec6..b37a0b7 100644
--- a/src/java/org/apache/cassandra/db/marshal/ShortType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ShortType.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class ShortType extends AbstractType<Short>
+public class ShortType extends NumberType<Short>
 {
     public static final ShortType instance = new ShortType();
 
@@ -91,4 +91,53 @@ public class ShortType extends AbstractType<Short>
     {
         return ShortSerializer.instance;
     }
+
+    @Override
+    public int valueLengthIfFixed()
+    {
+        return 2;
+    }
+
+    @Override
+    public short toShort(ByteBuffer value)
+    {
+        return ByteBufferUtil.toShort(value);
+    }
+
+    @Override
+    public int toInt(ByteBuffer value)
+    {
+        return toShort(value);
+    }
+
+    @Override
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((short) (leftType.toShort(left) + rightType.toShort(right)));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((short) (leftType.toShort(left) - rightType.toShort(right)));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((short) (leftType.toShort(left) * rightType.toShort(right)));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((short) (leftType.toShort(left) / rightType.toShort(right)));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((short) (leftType.toShort(left) % rightType.toShort(right)));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes((short) -toShort(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
index 36305a3..f8e58db 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
@@ -130,7 +130,7 @@ public class TimeUUIDType extends AbstractType<UUID>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 16;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/TimestampType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TimestampType.java b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
index 953ae1b..ae74e2f 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimestampType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
@@ -128,7 +128,7 @@ public class TimestampType extends AbstractType<Date>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 8;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/TupleType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java
index 60a63aa..71e946c 100644
--- a/src/java/org/apache/cassandra/db/marshal/TupleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java
@@ -101,6 +101,11 @@ public class TupleType extends AbstractType<ByteBuffer>
         return types;
     }
 
+    public boolean isTuple()
+    {
+        return true;
+    }
+
     public int compareCustom(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/UUIDType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UUIDType.java b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
index 9722a52..27e3360 100644
--- a/src/java/org/apache/cassandra/db/marshal/UUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
@@ -171,7 +171,7 @@ public class UUIDType extends AbstractType<UUID>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 16;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/UserType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index 176ab84..e47b7ac 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -86,6 +86,11 @@ public class UserType extends TupleType
         return true;
     }
 
+    public boolean isTuple()
+    {
+        return false;
+    }
+
     @Override
     public boolean isMultiCell()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/exceptions/OperationExecutionException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/OperationExecutionException.java b/src/java/org/apache/cassandra/exceptions/OperationExecutionException.java
new file mode 100644
index 0000000..4f9ffa4
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/OperationExecutionException.java
@@ -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.
+ */
+package org.apache.cassandra.exceptions;
+
+import java.util.List;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+/**
+ * Thrown when an operation problem has occured (e.g. division by zero with integer).
+ */
+public final class OperationExecutionException extends RequestExecutionException
+{
+
+    /**
+     * Creates a new <code>OperationExecutionException</code> for the specified operation.
+     *
+     * @param operator the operator
+     * @param argTypes the argument types
+     * @param e the original Exception
+     * @return a new <code>OperationExecutionException</code> for the specified operation
+     */
+    public static OperationExecutionException create(char operator, List<AbstractType<?>> argTypes, Exception e)
+    {
+        List<String> cqlTypes = AbstractType.asCQLTypeStringList(argTypes);
+        return new OperationExecutionException(String.format("the operation '%s %s %s' failed: %s",
+                                                             cqlTypes.get(0),
+                                                             operator,
+                                                             cqlTypes.get(1),
+                                                             e.getMessage()));
+    }
+
+    /**
+     * Creates an <code>OperationExecutionException</code> with the specified message.
+     * @param msg the error message
+     */
+    public OperationExecutionException(String msg)
+    {
+        super(ExceptionCode.FUNCTION_FAILURE, msg);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/serializers/ByteSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/ByteSerializer.java b/src/java/org/apache/cassandra/serializers/ByteSerializer.java
index 9d34fbc..8428e62 100644
--- a/src/java/org/apache/cassandra/serializers/ByteSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/ByteSerializer.java
@@ -28,12 +28,12 @@ public class ByteSerializer implements TypeSerializer<Byte>
 
     public Byte deserialize(ByteBuffer bytes)
     {
-        return bytes == null || bytes.remaining() == 0 ? null : bytes.get(bytes.position());
+        return bytes == null || bytes.remaining() == 0 ? null : ByteBufferUtil.toByte(bytes);
     }
 
     public ByteBuffer serialize(Byte value)
     {
-        return value == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : ByteBuffer.allocate(1).put(0, value);
+        return value == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : ByteBufferUtil.bytes(value);
     }
 
     public void validate(ByteBuffer bytes) throws MarshalException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
index fb1a9ec..b6f93bc 100644
--- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
+++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
@@ -435,6 +435,18 @@ public class ByteBufferUtil
         return bytes.getShort(bytes.position());
     }
 
+    /**
+     * Convert a byte buffer to a short.
+     * Does not change the byte buffer position.
+     *
+     * @param bytes byte buffer to convert to byte
+     * @return byte representation of the byte buffer
+     */
+    public static byte toByte(ByteBuffer bytes)
+    {
+        return bytes.get(bytes.position());
+    }
+
     public static long toLong(ByteBuffer bytes)
     {
         return bytes.getLong(bytes.position());
@@ -450,6 +462,11 @@ public class ByteBufferUtil
         return bytes.getDouble(bytes.position());
     }
 
+    public static ByteBuffer bytes(byte b)
+    {
+        return ByteBuffer.allocate(1).put(0, b);
+    }
+
     public static ByteBuffer bytes(short s)
     {
         return ByteBuffer.allocate(2).putShort(0, s);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 2a5afc2..598addd 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -1432,7 +1432,7 @@ public abstract class CQLTester
         return s;
     }
 
-    private static ByteBuffer makeByteBuffer(Object value, AbstractType type)
+    protected static ByteBuffer makeByteBuffer(Object value, AbstractType type)
     {
         if (value == null)
             return null;
@@ -1463,7 +1463,7 @@ public abstract class CQLTester
         return type.getString(bb);
     }
 
-    protected Object tuple(Object...values)
+    protected TupleValue tuple(Object...values)
     {
         return new TupleValue(values);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/test/unit/org/apache/cassandra/cql3/functions/OperationFctsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/functions/OperationFctsTest.java b/test/unit/org/apache/cassandra/cql3/functions/OperationFctsTest.java
new file mode 100644
index 0000000..6de5fdb
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/functions/OperationFctsTest.java
@@ -0,0 +1,744 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.exceptions.OperationExecutionException;
+
+public class OperationFctsTest extends CQLTester
+{
+    @Test
+    public void testSingleOperations() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a tinyint, b smallint, c int, d bigint, e float, f double, g varint, h decimal, PRIMARY KEY(a, b, c))");
+        execute("INSERT INTO %S (a, b, c, d, e, f, g, h) VALUES (1, 2, 3, 4, 5.5, 6.5, 7, 8.5)");
+
+        // Test additions
+        assertColumnNames(execute("SELECT a + a, b + a, c + a, d + a, e + a, f + a, g + a, h + a FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                          "a + a", "b + a", "c + a", "d + a", "e + a", "f + a", "g + a", "h + a");
+
+        assertRows(execute("SELECT a + a, b + a, c + a, d + a, e + a, f + a, g + a, h + a FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                   row((byte) 2, (short) 3, 4, 5L, 6.5F, 7.5, BigInteger.valueOf(8), BigDecimal.valueOf(9.5)));
+
+        assertRows(execute("SELECT a + b, b + b, c + b, d + b, e + b, f + b, g + b, h + b FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                   row((short) 3, (short) 4, 5, 6L, 7.5F, 8.5, BigInteger.valueOf(9), BigDecimal.valueOf(10.5)));
+
+        assertRows(execute("SELECT a + c, b + c, c + c, d + c, e + c, f + c, g + c, h + c FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                   row(4, 5, 6, 7L, 8.5F, 9.5, BigInteger.valueOf(10), BigDecimal.valueOf(11.5)));
+
+        assertRows(execute("SELECT a + d, b + d, c + d, d + d, e + d, f + d, g + d, h + d FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                   row(5L, 6L, 7L, 8L, 9.5, 10.5, BigInteger.valueOf(11), BigDecimal.valueOf(12.5)));
+
+        assertRows(execute("SELECT a + e, b + e, c + e, d + e, e + e, f + e, g + e, h + e FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                   row(6.5F, 7.5F, 8.5F, 9.5, 11.0F, 12.0, BigDecimal.valueOf(12.5), BigDecimal.valueOf(14.0)));
+
+        assertRows(execute("SELECT a + f, b + f, c + f, d + f, e + f, f + f, g + f, h + f FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                   row(7.5, 8.5, 9.5, 10.5, 12.0, 13.0, BigDecimal.valueOf(13.5), BigDecimal.valueOf(15.0)));
+
+        assertRows(execute("SELECT a + g, b + g, c + g, d + g, e + g, f + g, g + g, h + g FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                   row(BigInteger.valueOf(8),
+                       BigInteger.valueOf(9),
+                       BigInteger.valueOf(10),
+                       BigInteger.valueOf(11),
+                       BigDecimal.valueOf(12.5),
+                       BigDecimal.valueOf(13.5),
+                       BigInteger.valueOf(14),
+                       BigDecimal.valueOf(15.5)));
+
+        assertRows(execute("SELECT a + h, b + h, c + h, d + h, e + h, f + h, g + h, h + h FROM %s WHERE a = 1 AND b = 2 AND c = 1 + 2"),
+                   row(BigDecimal.valueOf(9.5),
+                       BigDecimal.valueOf(10.5),
+                       BigDecimal.valueOf(11.5),
+                       BigDecimal.valueOf(12.5),
+                       BigDecimal.valueOf(14.0),
+                       BigDecimal.valueOf(15.0),
+                       BigDecimal.valueOf(15.5),
+                       BigDecimal.valueOf(17.0)));
+
+        // Test substractions
+
+        assertColumnNames(execute("SELECT a - a, b - a, c - a, d - a, e - a, f - a, g - a, h - a FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                          "a - a", "b - a", "c - a", "d - a", "e - a", "f - a", "g - a", "h - a");
+
+        assertRows(execute("SELECT a - a, b - a, c - a, d - a, e - a, f - a, g - a, h - a FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                   row((byte) 0, (short) 1, 2, 3L, 4.5F, 5.5, BigInteger.valueOf(6), BigDecimal.valueOf(7.5)));
+
+        assertRows(execute("SELECT a - b, b - b, c - b, d - b, e - b, f - b, g - b, h - b FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                   row((short) -1, (short) 0, 1, 2L, 3.5F, 4.5, BigInteger.valueOf(5), BigDecimal.valueOf(6.5)));
+
+        assertRows(execute("SELECT a - c, b - c, c - c, d - c, e - c, f - c, g - c, h - c FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                   row(-2, -1, 0, 1L, 2.5F, 3.5, BigInteger.valueOf(4), BigDecimal.valueOf(5.5)));
+
+        assertRows(execute("SELECT a - d, b - d, c - d, d - d, e - d, f - d, g - d, h - d FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                   row(-3L, -2L, -1L, 0L, 1.5, 2.5, BigInteger.valueOf(3), BigDecimal.valueOf(4.5)));
+
+        assertRows(execute("SELECT a - e, b - e, c - e, d - e, e - e, f - e, g - e, h - e FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                   row(-4.5F, -3.5F, -2.5F, -1.5, 0.0F, 1.0, BigDecimal.valueOf(1.5), BigDecimal.valueOf(3.0)));
+
+        assertRows(execute("SELECT a - f, b - f, c - f, d - f, e - f, f - f, g - f, h - f FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                   row(-5.5, -4.5, -3.5, -2.5, -1.0, 0.0, BigDecimal.valueOf(0.5), BigDecimal.valueOf(2.0)));
+
+        assertRows(execute("SELECT a - g, b - g, c - g, d - g, e - g, f - g, g - g, h - g FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                   row(BigInteger.valueOf(-6),
+                       BigInteger.valueOf(-5),
+                       BigInteger.valueOf(-4),
+                       BigInteger.valueOf(-3),
+                       BigDecimal.valueOf(-1.5),
+                       BigDecimal.valueOf(-0.5),
+                       BigInteger.valueOf(0),
+                       BigDecimal.valueOf(1.5)));
+
+        assertRows(execute("SELECT a - h, b - h, c - h, d - h, e - h, f - h, g - h, h - h FROM %s WHERE a = 1 AND b = 2 AND c = 4 - 1"),
+                   row(BigDecimal.valueOf(-7.5),
+                       BigDecimal.valueOf(-6.5),
+                       BigDecimal.valueOf(-5.5),
+                       BigDecimal.valueOf(-4.5),
+                       BigDecimal.valueOf(-3.0),
+                       BigDecimal.valueOf(-2.0),
+                       BigDecimal.valueOf(-1.5),
+                       BigDecimal.valueOf(0.0)));
+
+        // Test multiplications
+
+        assertColumnNames(execute("SELECT a * a, b * a, c * a, d * a, e * a, f * a, g * a, h * a FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                          "a * a", "b * a", "c * a", "d * a", "e * a", "f * a", "g * a", "h * a");
+
+        assertRows(execute("SELECT a * a, b * a, c * a, d * a, e * a, f * a, g * a, h * a FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                   row((byte) 1, (short) 2, 3, 4L, 5.5F, 6.5, BigInteger.valueOf(7), new BigDecimal("8.50")));
+
+        assertRows(execute("SELECT a * b, b * b, c * b, d * b, e * b, f * b, g * b, h * b FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                   row((short) 2, (short) 4, 6, 8L, 11.0F, 13.0, BigInteger.valueOf(14), new BigDecimal("17.00")));
+
+        assertRows(execute("SELECT a * c, b * c, c * c, d * c, e * c, f * c, g * c, h * c FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                   row(3, 6, 9, 12L, 16.5F, 19.5, BigInteger.valueOf(21), new BigDecimal("25.50")));
+
+        assertRows(execute("SELECT a * d, b * d, c * d, d * d, e * d, f * d, g * d, h * d FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                   row(4L, 8L, 12L, 16L, 22.0, 26.0, BigInteger.valueOf(28), new BigDecimal("34.00")));
+
+        assertRows(execute("SELECT a * e, b * e, c * e, d * e, e * e, f * e, g * e, h * e FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                   row(5.5F, 11.0F, 16.5F, 22.0, 30.25F, 35.75, new BigDecimal("38.5"), new BigDecimal("46.75")));
+
+        assertRows(execute("SELECT a * f, b * f, c * f, d * f, e * f, f * f, g * f, h * f FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                   row(6.5, 13.0, 19.5, 26.0, 35.75, 42.25, new BigDecimal("45.5"), BigDecimal.valueOf(55.25)));
+
+        assertRows(execute("SELECT a * g, b * g, c * g, d * g, e * g, f * g, g * g, h * g FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                   row(BigInteger.valueOf(7),
+                       BigInteger.valueOf(14),
+                       BigInteger.valueOf(21),
+                       BigInteger.valueOf(28),
+                       new BigDecimal("38.5"),
+                       new BigDecimal("45.5"),
+                       BigInteger.valueOf(49),
+                       new BigDecimal("59.5")));
+
+        assertRows(execute("SELECT a * h, b * h, c * h, d * h, e * h, f * h, g * h, h * h FROM %s WHERE a = 1 AND b = 2 AND c = 3 * 1"),
+                   row(new BigDecimal("8.50"),
+                       new BigDecimal("17.00"),
+                       new BigDecimal("25.50"),
+                       new BigDecimal("34.00"),
+                       new BigDecimal("46.75"),
+                       new BigDecimal("55.25"),
+                       new BigDecimal("59.5"),
+                       new BigDecimal("72.25")));
+
+        // Test divisions
+
+        assertColumnNames(execute("SELECT a / a, b / a, c / a, d / a, e / a, f / a, g / a, h / a FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                          "a / a", "b / a", "c / a", "d / a", "e / a", "f / a", "g / a", "h / a");
+
+        assertRows(execute("SELECT a / a, b / a, c / a, d / a, e / a, f / a, g / a, h / a FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                   row((byte) 1, (short) 2, 3, 4L, 5.5F, 6.5, BigInteger.valueOf(7), new BigDecimal("8.5")));
+
+        assertRows(execute("SELECT a / b, b / b, c / b, d / b, e / b, f / b, g / b, h / b FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                   row((short) 0, (short) 1, 1, 2L, 2.75F, 3.25, BigInteger.valueOf(3), new BigDecimal("4.25")));
+
+        assertRows(execute("SELECT a / c, b / c, c / c, d / c, e / c, f / c, g / c, h / c FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                   row(0, 0, 1, 1L, 1.8333334F, 2.1666666666666665, BigInteger.valueOf(2), new BigDecimal("2.833333333333333333333333333333333")));
+
+        assertRows(execute("SELECT a / d, b / d, c / d, d / d, e / d, f / d, g / d, h / d FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                   row(0L, 0L, 0L, 1L, 1.375, 1.625, BigInteger.valueOf(1), new BigDecimal("2.125")));
+
+        assertRows(execute("SELECT a / e, b / e, c / e, d / e, e / e, f / e, g / e, h / e FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                   row(0.18181819F, 0.36363637F, 0.54545456F, 0.7272727272727273, 1.0F, 1.1818181818181819, new BigDecimal("1.272727272727272727272727272727273"), new BigDecimal("1.545454545454545454545454545454545")));
+
+        assertRows(execute("SELECT a / f, b / f, c / f, d / f, e / f, f / f, g / f, h / f FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                   row(0.15384615384615385, 0.3076923076923077, 0.46153846153846156, 0.6153846153846154, 0.8461538461538461, 1.0, new BigDecimal("1.076923076923076923076923076923077"), new BigDecimal("1.307692307692307692307692307692308")));
+
+        assertRows(execute("SELECT a / g, b / g, c / g, d / g, e / g, f / g, g / g, h / g FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                   row(BigInteger.valueOf(0),
+                       BigInteger.valueOf(0),
+                       BigInteger.valueOf(0),
+                       BigInteger.valueOf(0),
+                       new BigDecimal("0.7857142857142857142857142857142857"),
+                       new BigDecimal("0.9285714285714285714285714285714286"),
+                       BigInteger.valueOf(1),
+                       new BigDecimal("1.214285714285714285714285714285714")));
+
+        assertRows(execute("SELECT a / h, b / h, c / h, d / h, e / h, f / h, g / h, h / h FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"),
+                   row(new BigDecimal("0.1176470588235294117647058823529412"),
+                       new BigDecimal("0.2352941176470588235294117647058824"),
+                       new BigDecimal("0.3529411764705882352941176470588235"),
+                       new BigDecimal("0.4705882352941176470588235294117647"),
+                       new BigDecimal("0.6470588235294117647058823529411765"),
+                       new BigDecimal("0.7647058823529411764705882352941176"),
+                       new BigDecimal("0.8235294117647058823529411764705882"),
+                       new BigDecimal("1")));
+
+        // Test modulo operations
+
+        assertColumnNames(execute("SELECT a %% a, b %% a, c %% a, d %% a, e %% a, f %% a, g %% a, h %% a FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                          "a % a", "b % a", "c % a", "d % a", "e % a", "f % a", "g % a", "h % a");
+
+        assertRows(execute("SELECT a %% a, b %% a, c %% a, d %% a, e %% a, f %% a, g %% a, h %% a FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                   row((byte) 0, (short) 0, 0, 0L, 0.5F, 0.5, BigInteger.valueOf(0), new BigDecimal("0.5")));
+
+        assertRows(execute("SELECT a %% b, b %% b, c %% b, d %% b, e %% b, f %% b, g %% b, h %% b FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                   row((short) 1, (short) 0, 1, 0L, 1.5F, 0.5, BigInteger.valueOf(1), new BigDecimal("0.5")));
+
+        assertRows(execute("SELECT a %% c, b %% c, c %% c, d %% c, e %% c, f %% c, g %% c, h %% c FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                   row(1, 2, 0, 1L, 2.5F, 0.5, BigInteger.valueOf(1), new BigDecimal("2.5")));
+
+        assertRows(execute("SELECT a %% d, b %% d, c %% d, d %% d, e %% d, f %% d, g %% d, h %% d FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                   row(1L, 2L, 3L, 0L, 1.5, 2.5, BigInteger.valueOf(3), new BigDecimal("0.5")));
+
+        assertRows(execute("SELECT a %% e, b %% e, c %% e, d %% e, e %% e, f %% e, g %% e, h %% e FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                   row(1.0F, 2.0F, 3.0F, 4.0, 0.0F, 1.0, new BigDecimal("1.5"), new BigDecimal("3.0")));
+
+        assertRows(execute("SELECT a %% f, b %% f, c %% f, d %% f, e %% f, f %% f, g %% f, h %% f FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                   row(1.0, 2.0, 3.0, 4.0, 5.5, 0.0, new BigDecimal("0.5"), new BigDecimal("2.0")));
+
+        assertRows(execute("SELECT a %% g, b %% g, c %% g, d %% g, e %% g, f %% g, g %% g, h %% g FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                   row(BigInteger.valueOf(1),
+                       BigInteger.valueOf(2),
+                       BigInteger.valueOf(3),
+                       BigInteger.valueOf(4),
+                       new BigDecimal("5.5"),
+                       new BigDecimal("6.5"),
+                       BigInteger.valueOf(0),
+                       new BigDecimal("1.5")));
+
+        assertRows(execute("SELECT a %% h, b %% h, c %% h, d %% h, e %% h, f %% h, g %% h, h %% h FROM %s WHERE a = 1 AND b = 2 AND c = 23 %% 5"),
+                   row(new BigDecimal("1.0"),
+                       new BigDecimal("2.0"),
+                       new BigDecimal("3.0"),
+                       new BigDecimal("4.0"),
+                       new BigDecimal("5.5"),
+                       new BigDecimal("6.5"),
+                       new BigDecimal("7"),
+                       new BigDecimal("0.0")));
+
+        // Test negation
+
+        assertColumnNames(execute("SELECT -a, -b, -c, -d, -e, -f, -g, -h FROM %s WHERE a = 1 AND b = 2"),
+                          "-a", "-b", "-c", "-d", "-e", "-f", "-g", "-h");
+
+        assertRows(execute("SELECT -a, -b, -c, -d, -e, -f, -g, -h FROM %s WHERE a = 1 AND b = 2"),
+                   row((byte) -1, (short) -2, -3, -4L, -5.5F, -6.5, BigInteger.valueOf(-7), new BigDecimal("-8.5")));
+
+        // Test with null
+        execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", null, (byte) 1, (short) 2, 3);
+        assertRows(execute("SELECT a + d, b + d, c + d, d + d, e + d, f + d, g + d, h + d FROM %s WHERE a = 1 AND b = 2"),
+                   row(null, null, null, null, null, null, null, null));
+    }
+
+    @Test
+    public void testSingleOperationsWithLiterals() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, c1 tinyint, c2 smallint, v text, PRIMARY KEY(pk, c1, c2))");
+        execute("INSERT INTO %S (pk, c1, c2, v) VALUES (2, 2, 2, 'test')");
+
+        // There is only one function outputing tinyint
+        assertRows(execute("SELECT * FROM %s WHERE pk = 2 AND c1 = 1 + 1"),
+                   row(2, (byte) 2, (short) 2, "test"));
+
+        // As the operation can only be a sum between tinyints the expected type is tinyint
+        assertInvalidMessage("Expected 1 byte for a tinyint (4)",
+                             "SELECT * FROM %s WHERE pk = 2 AND c1 = 1 + ?", 1);
+
+        assertRows(execute("SELECT * FROM %s WHERE pk = 2 AND c1 = 1 + ?", (byte) 1),
+                   row(2, (byte) 2, (short) 2, "test"));
+
+        assertRows(execute("SELECT * FROM %s WHERE pk = 1 + 1 AND c1 = 2"),
+                   row(2, (byte) 2, (short) 2, "test"));
+
+        assertRows(execute("SELECT * FROM %s WHERE pk = 2 AND c1 = 2 AND c2 = 1 + 1"),
+                   row(2, (byte) 2, (short) 2, "test"));
+
+        assertRows(execute("SELECT * FROM %s WHERE pk = 2 AND c1 = 2 AND c2 = 1 * (1 + 1)"),
+                   row(2, (byte) 2, (short) 2, "test"));
+
+        // tinyint, smallint and int could be used there so we need to disambiguate
+        assertInvalidMessage("Ambiguous '+' operation: use type casts to disambiguate",
+                             "SELECT * FROM %s WHERE pk = ? + 1 AND c1 = 2", 1);
+
+        assertInvalidMessage("Ambiguous '+' operation: use type casts to disambiguate",
+                             "SELECT * FROM %s WHERE pk = 2 AND c1 = 2 AND c2 = 1 * (? + 1)", 1);
+
+        assertRows(execute("SELECT 1 + 1, v FROM %s WHERE pk = 2 AND c1 = 2"),
+                   row(2, "test"));
+
+        // As the output type is unknown the ? type cannot be determined
+        assertInvalidMessage("Ambiguous '+' operation: use type casts to disambiguate",
+                             "SELECT 1 + ?, v FROM %s WHERE pk = 2 AND c1 = 2", 1);
+
+        // As the prefered type for the constants is int, the returned type will be int
+        assertRows(execute("SELECT 100 + 50, v FROM %s WHERE pk = 2 AND c1 = 2"),
+                   row(150, "test"));
+
+        // As the output type is unknown the ? type cannot be determined
+        assertInvalidMessage("Ambiguous '+' operation: use type casts to disambiguate",
+                             "SELECT ? + 50, v FROM %s WHERE pk = 2 AND c1 = 2", 100);
+
+        createTable("CREATE TABLE %s (a tinyint, b smallint, c int, d bigint, e float, f double, g varint, h decimal, PRIMARY KEY(a, b))"
+                + " WITH CLUSTERING ORDER BY (b DESC)"); // Make sure we test with ReversedTypes
+        execute("INSERT INTO %S (a, b, c, d, e, f, g, h) VALUES (1, 2, 3, 4, 5.5, 6.5, 7, 8.5)");
+
+        // Test additions
+        assertColumnNames(execute("SELECT a + 1, b + 1, c + 1, d + 1, e + 1, f + 1, g + 1, h + 1 FROM %s WHERE a = 1 AND b = 2"),
+                          "a + 1", "b + 1", "c + 1", "d + 1", "e + 1", "f + 1", "g + 1", "h + 1");
+
+        assertRows(execute("SELECT a + 1, b + 1, c + 1, d + 1, e + 1, f + 1, g + 1, h + 1 FROM %s WHERE a = 1 AND b = 2"),
+                   row(2, 3, 4, 5L, 6.5F, 7.5, BigInteger.valueOf(8), BigDecimal.valueOf(9.5)));
+
+        assertRows(execute("SELECT 2 + a, 2 + b, 2 + c, 2 + d, 2 + e, 2 + f, 2 + g, 2 + h FROM %s WHERE a = 1 AND b = 2"),
+                   row(3, 4, 5, 6L, 7.5F, 8.5, BigInteger.valueOf(9), BigDecimal.valueOf(10.5)));
+
+        long bigInt = Integer.MAX_VALUE + 10L;
+
+        assertRows(execute("SELECT a + " + bigInt + ","
+                               + " b + " + bigInt + ","
+                               + " c + " + bigInt + ","
+                               + " d + " + bigInt + ","
+                               + " e + " + bigInt + ","
+                               + " f + " + bigInt + ","
+                               + " g + " + bigInt + ","
+                               + " h + " + bigInt + " FROM %s WHERE a = 1 AND b = 2"),
+                   row(1L + bigInt,
+                       2L + bigInt,
+                       3L + bigInt,
+                       4L + bigInt,
+                       5.5 + bigInt,
+                       6.5 + bigInt,
+                       BigInteger.valueOf(bigInt + 7),
+                       BigDecimal.valueOf(bigInt + 8.5)));
+
+        assertRows(execute("SELECT a + 5.5, b + 5.5, c + 5.5, d + 5.5, e + 5.5, f + 5.5, g + 5.5, h + 5.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(6.5, 7.5, 8.5, 9.5, 11.0, 12.0, BigDecimal.valueOf(12.5), BigDecimal.valueOf(14.0)));
+
+        assertRows(execute("SELECT a + 6.5, b + 6.5, c + 6.5, d + 6.5, e + 6.5, f + 6.5, g + 6.5, h + 6.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(7.5, 8.5, 9.5, 10.5, 12.0, 13.0, BigDecimal.valueOf(13.5), BigDecimal.valueOf(15.0)));
+
+        // Test substractions
+
+        assertColumnNames(execute("SELECT a - 1, b - 1, c - 1, d - 1, e - 1, f - 1, g - 1, h - 1 FROM %s WHERE a = 1 AND b = 2"),
+                          "a - 1", "b - 1", "c - 1", "d - 1", "e - 1", "f - 1", "g - 1", "h - 1");
+
+        assertRows(execute("SELECT a - 1, b - 1, c - 1, d - 1, e - 1, f - 1, g - 1, h - 1 FROM %s WHERE a = 1 AND b = 2"),
+                   row(0, 1, 2, 3L, 4.5F, 5.5, BigInteger.valueOf(6), BigDecimal.valueOf(7.5)));
+
+        assertRows(execute("SELECT a - 2, b - 2, c - 2, d - 2, e - 2, f - 2, g - 2, h - 2 FROM %s WHERE a = 1 AND b = 2"),
+                   row(-1, 0, 1, 2L, 3.5F, 4.5, BigInteger.valueOf(5), BigDecimal.valueOf(6.5)));
+
+        assertRows(execute("SELECT a - 3, b - 3, 3 - 3, d - 3, e - 3, f - 3, g - 3, h - 3 FROM %s WHERE a = 1 AND b = 2"),
+                   row(-2, -1, 0, 1L, 2.5F, 3.5, BigInteger.valueOf(4), BigDecimal.valueOf(5.5)));
+
+        assertRows(execute("SELECT a - " + bigInt + ","
+                               + " b - " + bigInt + ","
+                               + " c - " + bigInt + ","
+                               + " d - " + bigInt + ","
+                               + " e - " + bigInt + ","
+                               + " f - " + bigInt + ","
+                               + " g - " + bigInt + ","
+                               + " h - " + bigInt + " FROM %s WHERE a = 1 AND b = 2"),
+                   row(1L - bigInt,
+                       2L - bigInt,
+                       3L - bigInt,
+                       4L - bigInt,
+                       5.5 - bigInt,
+                       6.5 - bigInt,
+                       BigInteger.valueOf(7 - bigInt),
+                       BigDecimal.valueOf(8.5 - bigInt)));
+
+        assertRows(execute("SELECT a - 5.5, b - 5.5, c - 5.5, d - 5.5, e - 5.5, f - 5.5, g - 5.5, h - 5.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(-4.5, -3.5, -2.5, -1.5, 0.0, 1.0, BigDecimal.valueOf(1.5), BigDecimal.valueOf(3.0)));
+
+        assertRows(execute("SELECT a - 6.5, b - 6.5, c - 6.5, d - 6.5, e - 6.5, f - 6.5, g - 6.5, h - 6.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(-5.5, -4.5, -3.5, -2.5, -1.0, 0.0, BigDecimal.valueOf(0.5), BigDecimal.valueOf(2.0)));
+
+        // Test multiplications
+
+        assertColumnNames(execute("SELECT a * 1, b * 1, c * 1, d * 1, e * 1, f * 1, g * 1, h * 1 FROM %s WHERE a = 1 AND b = 2"),
+                          "a * 1", "b * 1", "c * 1", "d * 1", "e * 1", "f * 1", "g * 1", "h * 1");
+
+        assertRows(execute("SELECT a * 1, b * 1, c * 1, d * 1, e * 1, f * 1, g * 1, h * 1 FROM %s WHERE a = 1 AND b = 2"),
+                   row(1, 2, 3, 4L, 5.5F, 6.5, BigInteger.valueOf(7), new BigDecimal("8.50")));
+
+        assertRows(execute("SELECT a * 2, b * 2, c * 2, d * 2, e * 2, f * 2, g * 2, h * 2 FROM %s WHERE a = 1 AND b = 2"),
+                   row(2, 4, 6, 8L, 11.0F, 13.0, BigInteger.valueOf(14), new BigDecimal("17.00")));
+
+        assertRows(execute("SELECT a * 3, b * 3, c * 3, d * 3, e * 3, f * 3, g * 3, h * 3 FROM %s WHERE a = 1 AND b = 2"),
+                   row(3, 6, 9, 12L, 16.5F, 19.5, BigInteger.valueOf(21), new BigDecimal("25.50")));
+
+        assertRows(execute("SELECT a * " + bigInt + ","
+                            + " b * " + bigInt + ","
+                            + " c * " + bigInt + ","
+                            + " d * " + bigInt + ","
+                            + " e * " + bigInt + ","
+                            + " f * " + bigInt + ","
+                            + " g * " + bigInt + ","
+                            + " h * " + bigInt + " FROM %s WHERE a = 1 AND b = 2"),
+                               row(1L * bigInt,
+                                   2L * bigInt,
+                                   3L * bigInt,
+                                   4L * bigInt,
+                                   5.5 * bigInt,
+                                   6.5 * bigInt,
+                                   BigInteger.valueOf(7 * bigInt),
+                                   BigDecimal.valueOf(8.5 * bigInt)));
+
+        assertRows(execute("SELECT a * 5.5, b * 5.5, c * 5.5, d * 5.5, e * 5.5, f * 5.5, g * 5.5, h * 5.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(5.5, 11.0, 16.5, 22.0, 30.25, 35.75, new BigDecimal("38.5"), new BigDecimal("46.75")));
+
+        assertRows(execute("SELECT a * 6.5, b * 6.5, c * 6.5, d * 6.5, e * 6.5, 6.5 * f, g * 6.5, h * 6.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(6.5, 13.0, 19.5, 26.0, 35.75, 42.25, new BigDecimal("45.5"), BigDecimal.valueOf(55.25)));
+
+        // Test divisions
+
+        assertColumnNames(execute("SELECT a / 1, b / 1, c / 1, d / 1, e / 1, f / 1, g / 1, h / 1 FROM %s WHERE a = 1 AND b = 2"),
+                          "a / 1", "b / 1", "c / 1", "d / 1", "e / 1", "f / 1", "g / 1", "h / 1");
+
+        assertRows(execute("SELECT a / 1, b / 1, c / 1, d / 1, e / 1, f / 1, g / 1, h / 1 FROM %s WHERE a = 1 AND b = 2"),
+                   row(1, 2, 3, 4L, 5.5F, 6.5, BigInteger.valueOf(7), new BigDecimal("8.5")));
+
+        assertRows(execute("SELECT a / 2, b / 2, c / 2, d / 2, e / 2, f / 2, g / 2, h / 2 FROM %s WHERE a = 1 AND b = 2"),
+                   row(0, 1, 1, 2L, 2.75F, 3.25, BigInteger.valueOf(3), new BigDecimal("4.25")));
+
+        assertRows(execute("SELECT a / 3, b / 3, c / 3, d / 3, e / 3, f / 3, g / 3, h / 3 FROM %s WHERE a = 1 AND b = 2"),
+                   row(0, 0, 1, 1L, 1.8333334F, 2.1666666666666665, BigInteger.valueOf(2), new BigDecimal("2.833333333333333333333333333333333")));
+
+        assertRows(execute("SELECT a / " + bigInt + ","
+                + " b / " + bigInt + ","
+                + " c / " + bigInt + ","
+                + " d / " + bigInt + ","
+                + " e / " + bigInt + ","
+                + " f / " + bigInt + ","
+                + " g / " + bigInt + " FROM %s WHERE a = 1 AND b = 2"),
+                   row(1L / bigInt,
+                       2L / bigInt,
+                       3L / bigInt,
+                       4L / bigInt,
+                       5.5 / bigInt,
+                       6.5 / bigInt,
+                       BigInteger.valueOf(7).divide(BigInteger.valueOf(bigInt))));
+
+        assertRows(execute("SELECT a / 5.5, b / 5.5, c / 5.5, d / 5.5, e / 5.5, f / 5.5, g / 5.5, h / 5.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(0.18181818181818182, 0.36363636363636365, 0.5454545454545454, 0.7272727272727273, 1.0, 1.1818181818181819, new BigDecimal("1.272727272727272727272727272727273"), new BigDecimal("1.545454545454545454545454545454545")));
+
+        assertRows(execute("SELECT a / 6.5, b / 6.5, c / 6.5, d / 6.5, e / 6.5, f / 6.5, g / 6.5, h / 6.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(0.15384615384615385, 0.3076923076923077, 0.46153846153846156, 0.6153846153846154, 0.8461538461538461, 1.0, new BigDecimal("1.076923076923076923076923076923077"), new BigDecimal("1.307692307692307692307692307692308")));
+
+        // Test modulo operations
+
+        assertColumnNames(execute("SELECT a %% 1, b %% 1, c %% 1, d %% 1, e %% 1, f %% 1, g %% 1, h %% 1 FROM %s WHERE a = 1 AND b = 2"),
+                          "a % 1", "b % 1", "c % 1", "d % 1", "e % 1", "f % 1", "g % 1", "h % 1");
+
+        assertRows(execute("SELECT a %% 1, b %% 1, c %% 1, d %% 1, e %% 1, f %% 1, g %% 1, h %% 1 FROM %s WHERE a = 1 AND b = 2"),
+                   row(0, 0, 0, 0L, 0.5F, 0.5, BigInteger.valueOf(0), new BigDecimal("0.5")));
+
+        assertRows(execute("SELECT a %% 2, b %% 2, c %% 2, d %% 2, e %% 2, f %% 2, g %% 2, h %% 2 FROM %s WHERE a = 1 AND b = 2"),
+                   row(1, 0, 1, 0L, 1.5F, 0.5, BigInteger.valueOf(1), new BigDecimal("0.5")));
+
+        assertRows(execute("SELECT a %% 3, b %% 3, c %% 3, d %% 3, e %% 3, f %% 3, g %% 3, h %% 3 FROM %s WHERE a = 1 AND b = 2"),
+                   row(1, 2, 0, 1L, 2.5F, 0.5, BigInteger.valueOf(1), new BigDecimal("2.5")));
+
+        assertRows(execute("SELECT a %% " + bigInt + ","
+                            + " b %% " + bigInt + ","
+                            + " c %% " + bigInt + ","
+                            + " d %% " + bigInt + ","
+                            + " e %% " + bigInt + ","
+                            + " f %% " + bigInt + ","
+                            + " g %% " + bigInt + ","
+                            + " h %% " + bigInt + " FROM %s WHERE a = 1 AND b = 2"),
+                               row(1L % bigInt,
+                                   2L % bigInt,
+                                   3L % bigInt,
+                                   4L % bigInt,
+                                   5.5 % bigInt,
+                                   6.5 % bigInt,
+                                   BigInteger.valueOf(7 % bigInt),
+                                   BigDecimal.valueOf(8.5 % bigInt)));
+
+        assertRows(execute("SELECT a %% 5.5, b %% 5.5, c %% 5.5, d %% 5.5, e %% 5.5, f %% 5.5, g %% 5.5, h %% 5.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(1.0, 2.0, 3.0, 4.0, 0.0, 1.0, new BigDecimal("1.5"), new BigDecimal("3.0")));
+
+        assertRows(execute("SELECT a %% 6.5, b %% 6.5, c %% 6.5, d %% 6.5, e %% 6.5, f %% 6.5, g %% 6.5, h %% 6.5 FROM %s WHERE a = 1 AND b = 2"),
+                   row(1.0, 2.0, 3.0, 4.0, 5.5, 0.0, new BigDecimal("0.5"), new BigDecimal("2.0")));
+
+        assertRows(execute("SELECT a, b, 1 + 1, 2 - 1, 2 * 2, 2 / 1 , 2 %% 1, (int) -1 FROM %s WHERE a = 1 AND b = 2"),
+                   row((byte) 1, (short) 2, 2, 1, 4, 2, 0, -1));
+    }
+
+    @Test
+    public void testWithCounters() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b counter)");
+        execute("UPDATE %s SET b = b + 1 WHERE a = 1");
+        execute("UPDATE %s SET b = b + 1 WHERE a = 1");
+        assertRows(execute("SELECT b FROM %s WHERE a = 1"), row(2L));
+
+        assertRows(execute("SELECT b + (tinyint) 1,"
+                + " b + (smallint) 1,"
+                + " b + 1,"
+                + " b + (bigint) 1,"
+                + " b + (float) 1.5,"
+                + " b + 1.5,"
+                + " b + (varint) 1,"
+                + " b + (decimal) 1.5,"
+                + " b + b FROM %s WHERE a = 1"),
+                   row(3L, 3L, 3L, 3L, 3.5, 3.5, BigInteger.valueOf(3), new BigDecimal("3.5"), 4L));
+
+        assertRows(execute("SELECT b - (tinyint) 1,"
+                + " b - (smallint) 1,"
+                + " b - 1,"
+                + " b - (bigint) 1,"
+                + " b - (float) 1.5,"
+                + " b - 1.5,"
+                + " b - (varint) 1,"
+                + " b - (decimal) 1.5,"
+                + " b - b FROM %s WHERE a = 1"),
+                   row(1L, 1L, 1L, 1L, 0.5, 0.5, BigInteger.valueOf(1), new BigDecimal("0.5"), 0L));
+
+        assertRows(execute("SELECT b * (tinyint) 1,"
+                + " b * (smallint) 1,"
+                + " b * 1,"
+                + " b * (bigint) 1,"
+                + " b * (float) 1.5,"
+                + " b * 1.5,"
+                + " b * (varint) 1,"
+                + " b * (decimal) 1.5,"
+                + " b * b FROM %s WHERE a = 1"),
+                   row(2L, 2L, 2L, 2L, 3.0, 3.0, BigInteger.valueOf(2), new BigDecimal("3.00"), 4L));
+
+        assertRows(execute("SELECT b / (tinyint) 1,"
+                + " b / (smallint) 1,"
+                + " b / 1,"
+                + " b / (bigint) 1,"
+                + " b / (float) 0.5,"
+                + " b / 0.5,"
+                + " b / (varint) 1,"
+                + " b / (decimal) 0.5,"
+                + " b / b FROM %s WHERE a = 1"),
+                   row(2L, 2L, 2L, 2L, 4.0, 4.0, BigInteger.valueOf(2), new BigDecimal("4"), 1L));
+
+        assertRows(execute("SELECT b %% (tinyint) 1,"
+                + " b %% (smallint) 1,"
+                + " b %% 1,"
+                + " b %% (bigint) 1,"
+                + " b %% (float) 0.5,"
+                + " b %% 0.5,"
+                + " b %% (varint) 1,"
+                + " b %% (decimal) 0.5,"
+                + " b %% b FROM %s WHERE a = 1"),
+                   row(0L, 0L, 0L, 0L, 0.0, 0.0, BigInteger.valueOf(0), new BigDecimal("0.0"), 0L));
+
+        assertRows(execute("SELECT -b FROM %s WHERE a = 1"), row(-2L));
+    }
+
+    @Test
+    public void testPrecedenceAndParentheses() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY(a, b))");
+        execute("INSERT INTO %S (a, b, c, d) VALUES (2, 5, 25, 4)");
+
+        UntypedResultSet rs = execute("SELECT a - c / b + d FROM %s");
+        assertColumnNames(rs, "a - c / b + d");
+        assertRows(rs, row(1));
+
+        rs = execute("SELECT (c - b) / a + d FROM %s");
+        assertColumnNames(rs, "(c - b) / a + d");
+        assertRows(rs, row(14));
+
+        rs = execute("SELECT c / a / b FROM %s");
+        assertColumnNames(rs, "c / a / b");
+        assertRows(rs, row(2));
+
+        rs = execute("SELECT c / b / d FROM %s");
+        assertColumnNames(rs, "c / b / d");
+        assertRows(rs, row(1));
+
+        rs = execute("SELECT (c - a) %% d / a FROM %s");
+        assertColumnNames(rs, "(c - a) % d / a");
+        assertRows(rs, row(1));
+
+        rs = execute("SELECT (c - a) %% d / a + d FROM %s");
+        assertColumnNames(rs, "(c - a) % d / a + d");
+        assertRows(rs, row(5));
+
+        rs = execute("SELECT -(c - a) %% d / a + d FROM %s");
+        assertColumnNames(rs, "-(c - a) % d / a + d");
+        assertRows(rs, row(3));
+
+        rs = execute("SELECT (-c - a) %% d / a + d FROM %s");
+        assertColumnNames(rs, "(-c - a) % d / a + d");
+        assertRows(rs, row(3));
+
+        rs = execute("SELECT c - a %% d / a + d FROM %s");
+        assertColumnNames(rs, "c - a % d / a + d");
+        assertRows(rs, row(28));
+
+        rs = execute("SELECT (int)((c - a) %% d / (a + d)) FROM %s");
+        assertColumnNames(rs, "(int)((c - a) % d / (a + d))");
+        assertRows(rs, row(0));
+
+        // test with aliases
+        rs = execute("SELECT (int)((c - a) %% d / (a + d)) as result FROM %s");
+        assertColumnNames(rs, "result");
+        assertRows(rs, row(0));
+
+        rs = execute("SELECT c / a / b as divisions FROM %s");
+        assertColumnNames(rs, "divisions");
+        assertRows(rs, row(2));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = (int) ? / 2 - 5", 2, 20),
+                   row(2, 5, 25, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = (int) ? / (2 + 2)", 2, 20),
+                   row(2, 5, 25, 4));
+    }
+
+    @Test
+    public void testWithDivisionByZero() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a tinyint, b smallint, c int, d bigint, e float, f double, g varint, h decimal, PRIMARY KEY(a, b))");
+        execute("INSERT INTO %S (a, b, c, d, e, f, g, h) VALUES (0, 2, 3, 4, 5.5, 6.5, 7, 8.5)");
+
+        assertInvalidThrowMessage("the operation 'tinyint / tinyint' failed: / by zero",
+                                  OperationExecutionException.class,
+                                  "SELECT a / a FROM %s WHERE a = 0 AND b = 2");
+
+        assertInvalidThrowMessage("the operation 'smallint / tinyint' failed: / by zero",
+                                  OperationExecutionException.class,
+                                  "SELECT b / a FROM %s WHERE a = 0 AND b = 2");
+
+        assertInvalidThrowMessage("the operation 'int / tinyint' failed: / by zero",
+                                  OperationExecutionException.class,
+                                  "SELECT c / a FROM %s WHERE a = 0 AND b = 2");
+
+        assertInvalidThrowMessage("the operation 'bigint / tinyint' failed: / by zero",
+                                  OperationExecutionException.class,
+                                  "SELECT d / a FROM %s WHERE a = 0 AND b = 2");
+
+        assertInvalidThrowMessage("the operation 'smallint / smallint' failed: / by zero",
+                                  OperationExecutionException.class,
+                                  "SELECT a FROM %s WHERE a = 0 AND b = 10/0");
+
+        assertRows(execute("SELECT e / a FROM %s WHERE a = 0 AND b = 2"), row(Float.POSITIVE_INFINITY));
+        assertRows(execute("SELECT f / a FROM %s WHERE a = 0 AND b = 2"), row(Double.POSITIVE_INFINITY));
+
+        assertInvalidThrowMessage("the operation 'varint / tinyint' failed: BigInteger divide by zero",
+                                  OperationExecutionException.class,
+                                  "SELECT g / a FROM %s WHERE a = 0 AND b = 2");
+
+        assertInvalidThrowMessage("the operation 'decimal / tinyint' failed: Division by zero",
+                                  OperationExecutionException.class,
+                                  "SELECT h / a FROM %s WHERE a = 0 AND b = 2");
+    }
+
+    @Test
+    public void testWithNanAndInfinity() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b double, c decimal)");
+        assertInvalidMessage("Ambiguous '+' operation: use type casts to disambiguate",
+                             "INSERT INTO %S (a, b, c) VALUES (? + 1, ?, ?)", 0, Double.NaN, BigDecimal.valueOf(1));
+
+        execute("INSERT INTO %S (a, b, c) VALUES ((int) ? + 1, -?, ?)", 0, Double.NaN, BigDecimal.valueOf(1));
+
+        assertRows(execute("SELECT * FROM %s"), row(1, Double.NaN, BigDecimal.valueOf(1)));
+
+        assertRows(execute("SELECT a + NAN, b + 1 FROM %s"), row(Double.NaN, Double.NaN));
+        assertInvalidThrowMessage("the operation 'decimal + double' failed: A NaN cannot be converted into a decimal",
+                                  OperationExecutionException.class,
+                                  "SELECT c + NAN FROM %s");
+
+        assertRows(execute("SELECT a + (float) NAN, b + 1 FROM %s"), row(Float.NaN, Double.NaN));
+        assertInvalidThrowMessage("the operation 'decimal + float' failed: A NaN cannot be converted into a decimal",
+                                  OperationExecutionException.class,
+                                  "SELECT c + (float) NAN FROM %s");
+
+        execute("INSERT INTO %S (a, b, c) VALUES (?, ?, ?)", 1, Double.POSITIVE_INFINITY, BigDecimal.valueOf(1));
+        assertRows(execute("SELECT * FROM %s"), row(1, Double.POSITIVE_INFINITY, BigDecimal.valueOf(1)));
+
+        assertRows(execute("SELECT a + INFINITY, b + 1 FROM %s"), row(Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY));
+        assertInvalidThrowMessage("the operation 'decimal + double' failed: An infinite number cannot be converted into a decimal",
+                                  OperationExecutionException.class,
+                                  "SELECT c + INFINITY FROM %s");
+
+        assertRows(execute("SELECT a + (float) INFINITY, b + 1 FROM %s"), row(Float.POSITIVE_INFINITY, Double.POSITIVE_INFINITY));
+        assertInvalidThrowMessage("the operation 'decimal + float' failed: An infinite number cannot be converted into a decimal",
+                                  OperationExecutionException.class,
+                                  "SELECT c + (float) INFINITY FROM %s");
+
+        execute("INSERT INTO %S (a, b, c) VALUES (?, ?, ?)", 1, Double.NEGATIVE_INFINITY, BigDecimal.valueOf(1));
+        assertRows(execute("SELECT * FROM %s"), row(1, Double.NEGATIVE_INFINITY, BigDecimal.valueOf(1)));
+
+        assertRows(execute("SELECT a + -INFINITY, b + 1 FROM %s"), row(Double.NEGATIVE_INFINITY, Double.NEGATIVE_INFINITY));
+        assertInvalidThrowMessage("the operation 'decimal + double' failed: An infinite number cannot be converted into a decimal",
+                                  OperationExecutionException.class,
+                                  "SELECT c + -INFINITY FROM %s");
+
+        assertRows(execute("SELECT a + (float) -INFINITY, b + 1 FROM %s"), row(Float.NEGATIVE_INFINITY, Double.NEGATIVE_INFINITY));
+        assertInvalidThrowMessage("the operation 'decimal + float' failed: An infinite number cannot be converted into a decimal",
+                                  OperationExecutionException.class,
+                                  "SELECT c + (float) -INFINITY FROM %s");
+    }
+
+    @Test
+    public void testInvalidTypes() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b boolean, c text)");
+        execute("INSERT INTO %S (a, b, c) VALUES (?, ?, ?)", 1, true, "test");
+
+        assertInvalidMessage("the '+' operation is not supported between a and b", "SELECT a + b FROM %s");
+        assertInvalidMessage("the '+' operation is not supported between b and c", "SELECT b + c FROM %s");
+        assertInvalidMessage("the '+' operation is not supported between b and 1", "SELECT b + 1 FROM %s");
+        assertInvalidMessage("the '+' operation is not supported between 1 and b", "SELECT 1 + b FROM %s");
+        assertInvalidMessage("the '+' operation is not supported between b and NaN", "SELECT b + NaN FROM %s");
+        assertInvalidMessage("the '/' operation is not supported between a and b", "SELECT a / b FROM %s");
+        assertInvalidMessage("the '/' operation is not supported between b and c", "SELECT b / c FROM %s");
+        assertInvalidMessage("the '/' operation is not supported between b and 1", "SELECT b / 1 FROM %s");
+        assertInvalidMessage("the '/' operation is not supported between NaN and b", "SELECT NaN / b FROM %s");
+        assertInvalidMessage("the '/' operation is not supported between -Infinity and b", "SELECT -Infinity / b FROM %s");
+    }
+
+    @Test
+    public void testOverflow() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b tinyint, c smallint)");
+        execute("INSERT INTO %S (a, b, c) VALUES (?, ?, ?)", 1, (byte) 1, (short) 1);
+        assertRows(execute("SELECT a + (int) ?, b + (tinyint) ?, c + (smallint) ? FROM %s", 1, (byte) 1, (short) 1),
+                   row(2, (byte) 2,(short) 2));
+        assertRows(execute("SELECT a + (int) ?, b + (tinyint) ?, c + (smallint) ? FROM %s", Integer.MAX_VALUE, Byte.MAX_VALUE, Short.MAX_VALUE),
+                   row(Integer.MIN_VALUE, Byte.MIN_VALUE, Short.MIN_VALUE));
+    }
+}